You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by mm...@apache.org on 2022/08/11 22:19:12 UTC

[ignite] branch master updated: IGNITE-15759 Remove LOCAL caches (#10157)

This is an automated email from the ASF dual-hosted git repository.

mmuzaf pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 01a7d075a5f IGNITE-15759 Remove LOCAL caches (#10157)
01a7d075a5f is described below

commit 01a7d075a5f48016511f6a754538201f12aff4f7
Author: Maxim Muzafarov <ma...@gmail.com>
AuthorDate: Fri Aug 12 01:19:04 2022 +0300

    IGNITE-15759 Remove LOCAL caches (#10157)
---
 .../processors/query/calcite/exec/IndexScan.java   |    5 +-
 .../processors/query/calcite/exec/TableScan.java   |    5 +-
 .../ClientAbstractMultiThreadedSelfTest.java       |   64 +-
 .../client/ClientDefaultCacheSelfTest.java         |   21 +-
 .../ClientAbstractMultiNodeSelfTest.java           |   55 +-
 .../client/integration/ClientAbstractSelfTest.java |   25 +-
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java    |  173 ---
 .../rest/RestBinaryProtocolSelfTest.java           |   17 +-
 .../rest/RestMemcacheProtocolSelfTest.java         |   17 +-
 .../rest/TaskCommandHandlerSelfTest.java           |   36 +-
 .../ignite/jdbc/JdbcLocalCachesSelfTest.java       |  141 --
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |    3 -
 .../JdbcThinCacheToJdbcDataTypesCoverageTest.java  |    5 +-
 .../JdbcThinJdbcToCacheDataTypesCoverageTest.java  |    9 +-
 .../clients/src/test/resources/spring-cache.xml    |   11 -
 .../src/test/resources/spring-server-node.xml      |   26 -
 .../src/test/resources/spring-server-ssl-node.xml  |   12 -
 .../ignite/compatibility/cache/LocalCacheTest.java |  136 --
 .../ignite/compatibility/cache/package-info.java   |   22 -
 .../IgniteCompatibilityBasicTestSuite.java         |    2 -
 .../java/org/apache/ignite/cache/CacheMode.java    |   73 +-
 .../org/apache/ignite/cache/CachePeekMode.java     |    3 -
 .../apache/ignite/cache/CacheRebalanceMode.java    |    3 +-
 .../org/apache/ignite/internal/IgnitionEx.java     |    2 +-
 .../internal/client/GridClientCacheMode.java       |    3 -
 .../ignite/internal/client/thin/ClientUtils.java   |    5 +-
 .../internal/client/thin/TcpIgniteClient.java      |    5 +-
 .../ignite/internal/jdbc2/JdbcBatchUpdateTask.java |    9 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java       |    2 +-
 .../ignite/internal/jdbc2/JdbcStatement.java       |    2 +-
 .../affinity/GridAffinityAssignmentCache.java      |   26 +-
 .../processors/affinity/GridAffinityProcessor.java |    5 -
 .../processors/affinity/LocalAffinityFunction.java |   78 -
 .../cache/CacheAffinitySharedManager.java          |   40 +-
 .../processors/cache/CacheGroupContext.java        |   39 +-
 .../processors/cache/CacheMetricsImpl.java         |   52 +-
 .../processors/cache/ClusterCachesInfo.java        |  170 +--
 .../cache/GatewayProtectedCacheProxy.java          |    3 -
 .../processors/cache/GridCacheAdapter.java         |  221 +--
 .../processors/cache/GridCacheAffinityManager.java |   30 -
 .../processors/cache/GridCacheAttributes.java      |    4 +-
 .../processors/cache/GridCacheContext.java         |   16 -
 .../cache/GridCacheDeploymentManager.java          |   13 -
 .../processors/cache/GridCacheEntryEx.java         |   42 -
 .../processors/cache/GridCacheMapEntry.java        |  358 -----
 .../processors/cache/GridCacheMvccManager.java     |    4 +-
 .../cache/GridCachePartitionExchangeManager.java   |   93 +-
 .../processors/cache/GridCacheProcessor.java       |  120 +-
 .../processors/cache/GridCacheTtlManager.java      |    2 +-
 .../internal/processors/cache/GridCacheUtils.java  |   22 +-
 .../cache/IgniteCacheOffheapManagerImpl.java       |   42 +-
 .../processors/cache/IgniteCacheProxyImpl.java     |   18 +-
 .../processors/cache/IgniteInternalCache.java      |    6 +-
 .../cache/ValidationOnNodeJoinUtils.java           |   17 +-
 .../internal/processors/cache/WalStateManager.java |    7 +-
 .../cache/affinity/GridCacheAffinityImpl.java      |    7 +-
 .../datastructures/CacheDataStructuresManager.java |  117 +-
 .../dht/GridDhtTopologyFutureAdapter.java          |    3 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java    |    6 +-
 .../dht/GridDhtTxQueryEnlistFuture.java            |    2 +-
 .../preloader/GridDhtPartitionsExchangeFuture.java |   74 +-
 .../PartitionDefferedDeleteQueueCleanupTask.java   |    2 +-
 ...dNearOptimisticSerializableTxPrepareFuture.java |   46 +-
 .../near/GridNearOptimisticTxPrepareFuture.java    |   14 +-
 .../GridNearOptimisticTxPrepareFutureAdapter.java  |    4 +-
 .../near/GridNearPessimisticTxPrepareFuture.java   |    9 +-
 .../cache/distributed/near/GridNearTxLocal.java    |  189 +--
 .../processors/cache/local/GridLocalCache.java     |  263 ----
 .../cache/local/GridLocalCacheEntry.java           |  382 -----
 .../cache/local/GridLocalLockCallback.java         |   39 -
 .../cache/local/GridLocalLockFuture.java           |  556 -------
 .../cache/local/atomic/GridLocalAtomicCache.java   | 1580 --------------------
 .../GridCacheDatabaseSharedManager.java            |   12 +-
 .../cache/persistence/GridCacheOffheapManager.java |   50 +-
 .../persistence/checkpoint/CheckpointWorkflow.java |    2 +-
 .../UpgradePendingTreeToPerPartitionTask.java      |    2 +-
 .../query/GridCacheDistributedQueryManager.java    |    9 -
 .../cache/query/GridCacheLocalQueryManager.java    |  131 --
 .../cache/query/GridCacheQueryAdapter.java         |   30 +-
 .../cache/query/GridCacheQueryManager.java         |   21 +-
 .../continuous/CacheContinuousQueryHandler.java    |   18 +-
 .../continuous/CacheContinuousQueryManager.java    |   18 +-
 .../cache/query/jdbc/GridCacheQueryJdbcTask.java   |    4 +-
 .../IgniteTxImplicitSingleStateImpl.java           |    4 +-
 .../cache/transactions/IgniteTxLocalAdapter.java   |    8 +-
 .../cache/transactions/IgniteTxStateImpl.java      |   14 +-
 .../processors/cache/verify/CacheInfo.java         |    4 +-
 .../cache/verify/VerifyBackupPartitionsTask.java   |    2 +-
 .../cache/verify/VerifyBackupPartitionsTaskV2.java |    4 +-
 .../continuous/GridContinuousProcessor.java        |    6 +-
 .../processors/datastreamer/DataStreamerImpl.java  |   67 +-
 .../datastructures/GridCacheSetImpl.java           |    2 +-
 .../datastructures/GridSetQueryPredicate.java      |    2 +-
 .../internal/processors/job/GridJobProcessor.java  |    2 +-
 .../processors/platform/client/ClientStatus.java   |    3 +
 .../cache/ClientCacheConfigurationSerializer.java  |   12 +-
 .../ClientCacheCreateWithConfigurationRequest.java |   11 +
 ...ntCacheGetOrCreateWithConfigurationRequest.java |    3 +
 .../ClientAtomicLongCreateRequest.java             |    2 +-
 .../ClientIgniteSetGetOrCreateRequest.java         |    2 +-
 .../platform/utils/PlatformConfigurationUtils.java |    9 +-
 .../processors/query/GridQueryProcessor.java       |   12 +-
 .../ignite/internal/sql/SqlCommandProcessor.java   |   19 -
 .../ignite/internal/visor/cache/VisorCache.java    |    4 +-
 .../visor/cache/VisorCacheAggregatedMetrics.java   |    4 +-
 .../visor/cache/VisorCacheConfiguration.java       |    5 +-
 .../internal/visor/cache/VisorCacheMetrics.java    |    4 +-
 ...orFindAndDeleteGarbageInPersistenceClosure.java |    2 +-
 .../visor/node/VisorAtomicConfiguration.java       |    5 +-
 .../visor/verify/ValidateIndexesClosure.java       |    2 +-
 .../main/resources/META-INF/classnames.properties  |   10 -
 modules/core/src/test/config/example-cache.xml     |   14 -
 .../core/src/test/config/load/cache-benchmark.xml  |    9 -
 .../test/config/load/cache-client-benchmark.xml    |    2 +-
 modules/core/src/test/config/spring-cache-swap.xml |    2 +-
 .../test/config/websession/example-cache-base.xml  |   14 -
 .../affinity/local/LocalAffinityFunctionTest.java  |   69 -
 .../GridCacheHashMapPutAllWarningsTest.java        |   56 -
 .../processors/cache/CacheClientStoreSelfTest.java |   51 -
 .../cache/CacheDeferredDeleteSanitySelfTest.java   |   19 -
 .../cache/CacheGetEntryAbstractTest.java           |   56 +-
 ...eGetEntryPessimisticRepeatableReadSelfTest.java |   18 -
 ...InterceptorPartitionCounterLocalSanityTest.java |  692 ---------
 .../cache/CacheLockCandidatesThreadTest.java       |    8 -
 .../cache/CacheMetricsEntitiesCountTest.java       |    9 -
 .../cache/CacheOffheapMapEntrySelfTest.java        |   10 -
 ...CacheReadThroughLocalAtomicRestartSelfTest.java |   32 -
 .../CacheReadThroughLocalRestartSelfTest.java      |   48 -
 .../processors/cache/CacheRebalancingSelfTest.java |   23 -
 .../cache/CacheStopAndDestroySelfTest.java         |  152 --
 .../cache/GridCacheAbstractFullApiSelfTest.java    |   14 +-
 .../cache/GridCacheClearLocallySelfTest.java       |  119 +-
 .../cache/GridCacheConcurrentMapSelfTest.java      |   16 +-
 .../GridCacheConfigurationConsistencySelfTest.java |   64 +-
 .../GridCacheConfigurationValidationSelfTest.java  |   13 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java    |   28 -
 .../GridCacheFastNodeLeftForTransactionTest.java   |    2 +-
 .../GridCacheInterceptorAbstractSelfTest.java      |   62 +-
 .../GridCacheInterceptorLocalAtomicSelfTest.java   |   44 -
 ...cheInterceptorLocalAtomicWithStoreSelfTest.java |   28 -
 .../cache/GridCacheInterceptorLocalSelfTest.java   |   60 -
 ...GridCacheInterceptorLocalWithStoreSelfTest.java |   45 -
 .../GridCacheLocalTxStoreExceptionSelfTest.java    |   45 -
 .../cache/GridCacheMvccManagerSelfTest.java        |   10 -
 .../cache/GridCacheObjectToStringSelfTest.java     |   20 -
 .../cache/GridCacheOnCopyFlagLocalSelfTest.java    |   41 -
 .../cache/GridCacheReferenceCleanupSelfTest.java   |   77 -
 .../processors/cache/GridCacheReloadSelfTest.java  |   14 -
 .../processors/cache/GridCacheSlowTxWarnTest.java  |   48 +-
 .../processors/cache/GridCacheTestEntryEx.java     |   24 -
 .../cache/GridCacheTtlManagerEvictionSelfTest.java |    8 -
 .../cache/GridCacheTtlManagerSelfTest.java         |   10 -
 ...dLocalCacheStoreManagerDeserializationTest.java |  103 --
 ...acheAtomicConcurrentUnorderedUpdateAllTest.java |    5 +-
 .../cache/IgniteCacheAtomicLocalInvokeTest.java    |   50 -
 .../cache/IgniteCacheAtomicLocalPeekModesTest.java |   44 -
 .../IgniteCacheAtomicLocalStoreValueTest.java      |   50 -
 .../IgniteCacheAtomicLocalWithStoreInvokeTest.java |   31 -
 .../IgniteCacheConfigVariationsFullApiTest.java    |   11 +-
 .../IgniteCacheEntryListenerAbstractTest.java      |    8 -
 .../IgniteCacheEntryListenerAtomicLocalTest.java   |   50 -
 .../cache/IgniteCacheEntryListenerTxLocalTest.java |   58 -
 .../processors/cache/IgniteCacheGroupsTest.java    |  210 +--
 .../cache/IgniteCacheInterceptorSelfTestSuite.java |    8 -
 ...IgniteCacheInvokeReadThroughSingleNodeTest.java |   27 -
 .../cache/IgniteCachePeekModesAbstractTest.java    |  435 +-----
 .../cache/IgniteCachePutStackOverflowSelfTest.java |    8 -
 .../cache/IgniteCacheStoreValueAbstractTest.java   |    5 +-
 .../cache/IgniteCacheTxLocalInvokeTest.java        |   58 -
 .../cache/IgniteCacheTxLocalPeekModesTest.java     |   60 -
 .../cache/IgniteCacheTxLocalStoreValueTest.java    |   58 -
 .../cache/IgniteTxExceptionAbstractSelfTest.java   |    5 -
 .../IgniteTxStoreExceptionAbstractSelfTest.java    |    5 -
 .../cache/WalModeChangeAbstractSelfTest.java       |   41 -
 .../GridCacheBinaryObjectsAbstractSelfTest.java    |    9 +-
 .../GridCacheBinaryObjectsAtomicLocalSelfTest.java |   32 -
 .../GridCacheBinaryObjectsLocalOnheapSelfTest.java |   32 -
 .../local/GridCacheBinaryObjectsLocalSelfTest.java |   51 -
 .../GridCacheSetAbstractSelfTest.java              |   10 +-
 .../IgniteDataStructuresTestUtils.java             |   10 -
 .../datastructures/IgniteLockAbstractSelfTest.java |    5 -
 .../IgniteSemaphoreAbstractSelfTest.java           |    6 -
 .../GridCacheLocalAtomicQueueApiSelfTest.java      |   32 -
 .../local/GridCacheLocalAtomicSetSelfTest.java     |   45 -
 .../local/GridCacheLocalQueueApiSelfTest.java      |   40 -
 .../local/GridCacheLocalSequenceApiSelfTest.java   |   33 -
 .../local/GridCacheLocalSetSelfTest.java           |   45 -
 .../local/IgniteLocalAtomicLongApiSelfTest.java    |   33 -
 .../local/IgniteLocalCountDownLatchSelfTest.java   |   97 --
 .../local/IgniteLocalLockSelfTest.java             |  112 --
 .../local/IgniteLocalSemaphoreSelfTest.java        |  100 --
 .../GridCacheTransformEventSelfTest.java           |   89 +-
 .../IgniteAbstractTxSuspendResumeTest.java         |   14 +-
 .../distributed/IgniteCacheTxIteratorSelfTest.java |    3 +-
 .../IgniteOptimisticTxSuspendResumeTest.java       |    5 -
 .../GridCacheDhtPreloadMultiThreadedSelfTest.java  |    3 -
 .../near/GridCacheGetStoreErrorSelfTest.java       |    8 -
 .../GridCacheRebalancingSyncSelfTest.java          |    4 +-
 .../cache/eviction/EvictionAbstractTest.java       |  105 --
 .../EvictionPolicyFactoryAbstractTest.java         |  108 --
 ...CacheConcurrentEvictionConsistencySelfTest.java |   35 +-
 .../GridCacheConcurrentEvictionsSelfTest.java      |   38 +-
 .../GridCacheEmptyEntriesLocalSelfTest.java        |   51 -
 .../eviction/GridCacheEvictionFilterSelfTest.java  |   15 +-
 .../GridCacheEvictionLockUnlockSelfTest.java       |   13 -
 .../PageEvictionPagesRecyclingAndReusingTest.java  |   25 -
 .../paged/PageEvictionReadThroughTest.java         |   26 -
 .../eviction/paged/PageEvictionTouchOrderTest.java |   25 -
 .../IgniteCacheAtomicLocalExpiryPolicyTest.java    |   53 -
 ...niteCacheAtomicLocalOnheapExpiryPolicyTest.java |   32 -
 .../expiry/IgniteCacheExpiryPolicyTestSuite.java   |    3 -
 .../expiry/IgniteCacheTxLocalExpiryPolicyTest.java |   50 -
 .../IgniteCacheAtomicLocalLoadAllTest.java         |   50 -
 ...iteCacheAtomicLocalNoLoadPreviousValueTest.java |   50 -
 .../IgniteCacheAtomicLocalNoReadThroughTest.java   |   50 -
 .../IgniteCacheAtomicLocalNoWriteThroughTest.java  |   50 -
 .../integration/IgniteCacheTxLocalLoadAllTest.java |   58 -
 .../IgniteCacheTxLocalNoLoadPreviousValueTest.java |   66 -
 .../IgniteCacheTxLocalNoReadThroughTest.java       |   66 -
 .../IgniteCacheTxLocalNoWriteThroughTest.java      |   66 -
 ...GridCacheAtomicLocalMetricsNoStoreSelfTest.java |   36 -
 .../local/GridCacheAtomicLocalMetricsSelfTest.java |   57 -
 ...GridCacheAtomicLocalTckMetricsSelfTestImpl.java |  188 ---
 .../local/GridCacheDaemonNodeLocalSelfTest.java    |   41 -
 .../GridCacheLocalAtomicBasicStoreSelfTest.java    |   32 -
 .../local/GridCacheLocalAtomicFullApiSelfTest.java |   42 -
 ...cheLocalAtomicGetAndTransformStoreSelfTest.java |   33 -
 ...cheLocalAtomicMetricsNoReadThroughSelfTest.java |   48 -
 ...idCacheLocalAtomicWithGroupFullApiSelfTest.java |   34 -
 .../local/GridCacheLocalBasicApiSelfTest.java      |   52 -
 ...dCacheLocalBasicStoreMultithreadedSelfTest.java |   31 -
 .../local/GridCacheLocalBasicStoreSelfTest.java    |   49 -
 .../GridCacheLocalByteArrayValuesSelfTest.java     |  214 ---
 .../cache/local/GridCacheLocalEventSelfTest.java   |   56 -
 .../local/GridCacheLocalEvictionEventSelfTest.java |   55 -
 ...GridCacheLocalFullApiMultithreadedSelfTest.java |   38 -
 .../cache/local/GridCacheLocalFullApiSelfTest.java |  160 --
 ...GridCacheLocalGetAndTransformStoreSelfTest.java |   49 -
 .../local/GridCacheLocalIsolatedNodesSelfTest.java |  123 --
 .../local/GridCacheLocalIteratorsSelfTest.java     |   51 -
 .../cache/local/GridCacheLocalLoadAllSelfTest.java |  122 --
 .../cache/local/GridCacheLocalLockSelfTest.java    |  335 -----
 .../cache/local/GridCacheLocalMetricsSelfTest.java |   67 -
 .../local/GridCacheLocalMultithreadedSelfTest.java |  369 -----
 .../local/GridCacheLocalTxExceptionSelfTest.java   |   46 -
 .../GridCacheLocalTxMultiThreadedSelfTest.java     |  100 --
 .../cache/local/GridCacheLocalTxReadTest.java      |   33 -
 .../GridCacheLocalTxSingleThreadedSelfTest.java    |   95 --
 .../local/GridCacheLocalTxTimeoutSelfTest.java     |  185 ---
 .../GridCacheLocalWithGroupFullApiSelfTest.java    |   34 -
 .../mvcc/CacheMvccConfigurationValidationTest.java |   19 -
 .../IgnitePdsDestroyCacheAbstractTest.java         |   13 +-
 .../persistence/IgnitePdsDestroyCacheTest.java     |   21 +-
 .../persistence/IgnitePdsDynamicCacheTest.java     |   11 -
 .../db/IgnitePdsPartitionPreloadTest.java          |   56 +-
 .../cache/persistence/db/IgnitePdsWithTtlTest.java |   12 -
 .../pagemem/PageMemoryLazyAllocationTest.java      |   26 -
 .../PageMemoryLazyAllocationWithPDSTest.java       |    8 -
 .../cache/query/CacheScanQueryFailoverTest.java    |   17 -
 .../CacheContinuousQueryCounterAbstractTest.java   |    5 -
 .../CacheContinuousQueryExecuteInPrimaryTest.java  |   37 -
 ...acheContinuousWithTransformerLocalSelfTest.java |   29 -
 .../GridCacheContinuousQueryAbstractSelfTest.java  |    8 -
 ...ridCacheContinuousQueryLocalAtomicSelfTest.java |   32 -
 .../GridCacheContinuousQueryLocalSelfTest.java     |   37 -
 .../store/GridCacheWriteBehindStoreLocalTest.java  |   46 -
 .../TxLocalDhtMixedCacheModesTest.java             |   86 --
 .../TxPessimisticDeadlockDetectionTest.java        |   28 -
 ...sionedEntryLocalAtomicSwapDisabledSelfTest.java |   46 -
 ...heVersionedEntryLocalTransactionalSelfTest.java |   57 -
 .../DataStreamProcessorMvccSelfTest.java           |    8 -
 .../datastreamer/DataStreamProcessorSelfTest.java  |   23 -
 .../query/ScanQueriesTopologyMappingTest.java      |   29 -
 .../cache/GridCacheCommandHandlerSelfTest.java     |    2 +-
 .../top/CacheTopologyCommandHandlerTest.java       |    2 +-
 .../loadtests/hashmap/GridCacheTestContext.java    |    5 +-
 .../ignite/testframework/MvccFeatureChecker.java   |   18 -
 .../junits/common/GridCommonAbstractTest.java      |   13 +-
 .../testsuites/IgniteBinaryCacheTestSuite.java     |    2 -
 .../testsuites/IgniteBinaryObjectsTestSuite.java   |    6 -
 ...niteCacheDataStructuresBinarySelfTestSuite.java |    4 -
 .../IgniteCacheDataStructuresSelfTestSuite.java    |   19 -
 .../IgniteCacheEvictionSelfTestSuite.java          |    2 -
 .../IgniteCacheFullApiSelfTestSuite.java           |   10 -
 .../IgniteCacheIteratorsSelfTestSuite.java         |    2 -
 .../IgniteCacheMetricsSelfTestSuite.java           |   10 -
 .../testsuites/IgniteCacheMvccTestSuite1.java      |   10 -
 .../testsuites/IgniteCacheMvccTestSuite2.java      |    8 -
 .../testsuites/IgniteCacheMvccTestSuite3.java      |    6 -
 .../testsuites/IgniteCacheMvccTestSuite4.java      |   16 -
 .../testsuites/IgniteCacheMvccTestSuite5.java      |    2 -
 .../testsuites/IgniteCacheMvccTestSuite6.java      |    4 -
 .../testsuites/IgniteCacheMvccTestSuite8.java      |    8 -
 .../ignite/testsuites/IgniteCacheTestSuite.java    |   10 -
 .../ignite/testsuites/IgniteCacheTestSuite10.java  |    4 -
 .../ignite/testsuites/IgniteCacheTestSuite11.java  |    2 -
 .../ignite/testsuites/IgniteCacheTestSuite2.java   |   36 +-
 .../ignite/testsuites/IgniteCacheTestSuite3.java   |    4 -
 .../ignite/testsuites/IgniteCacheTestSuite4.java   |   34 -
 .../ignite/testsuites/IgniteCacheTestSuite6.java   |    3 -
 .../IgniteCacheWriteBehindTestSuite.java           |    2 -
 .../org/apache/ignite/util/TestStorageUtils.java   |    2 -
 .../test/webapp/META-INF/ignite-webapp-config.xml  |   11 -
 .../processors/query/h2/CommandProcessor.java      |    4 -
 .../processors/query/h2/IgniteH2Indexing.java      |    2 +-
 .../h2/twostep/PartitionReservationManager.java    |    2 +-
 .../query/h2/twostep/ReducePartitionMapper.java    |   17 +-
 .../CacheLocalQueryDetailMetricsSelfTest.java      |   33 -
 .../cache/CacheLocalQueryMetricsSelfTest.java      |   33 -
 .../IgniteCacheAbstractFieldsQuerySelfTest.java    |    5 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java    |    3 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java     |    9 +-
 .../index/DynamicIndexAbstractBasicSelfTest.java   |   29 -
 .../cache/index/DynamicIndexAbstractSelfTest.java  |   17 -
 .../cache/index/H2DynamicTableSelfTest.java        |   21 -
 .../local/IgniteCacheLocalAtomicQuerySelfTest.java |   32 -
 .../local/IgniteCacheLocalFieldsQuerySelfTest.java |   53 -
 ...niteCacheLocalQueryCancelOrTimeoutSelfTest.java |  192 ---
 ...gniteCacheLocalQueryDefaultTimeoutSelfTest.java |  161 --
 .../cache/local/IgniteCacheLocalQuerySelfTest.java |  152 --
 .../persistence/db/wal/IgniteWalRecoveryTest.java  |   49 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java        |    5 -
 .../cache/ttl/CacheTtlAtomicLocalSelfTest.java     |   34 -
 .../ttl/CacheTtlTransactionalLocalSelfTest.java    |   34 -
 .../query/IgniteSqlEntryCacheModeAgnosticTest.java |   14 +-
 .../query/IgniteSqlNotNullConstraintTest.java      |   28 +-
 .../query/SqlQueriesTopologyMappingTest.java       |   30 -
 .../ignite/sqltests/SqlDataTypesCoverageTests.java |   10 +-
 .../IgniteBinaryCacheQueryTestSuite.java           |    4 -
 .../IgniteBinaryCacheQueryTestSuite3.java          |    8 -
 .../IgniteBinaryCacheQueryTestSuite4.java          |    3 -
 .../testsuites/IgniteCacheQuerySelfTestSuite5.java |    4 -
 .../testsuites/IgniteCacheQuerySelfTestSuite6.java |    2 -
 .../IgniteCacheWithIndexingTestSuite.java          |    4 -
 .../cache-native-persistence-test-default.xml      |   12 -
 .../cpp/core-test/config/cache-test-default.xml    |   12 -
 .../core-test/config/compute-server0-default.xml   |   12 -
 .../core-test/config/compute-server1-default.xml   |   12 -
 .../cpp/thin-client-test/config/cache-default.xml  |   12 -
 .../cpp/thin-client-test/src/cache_client_test.cpp |   68 +-
 .../Cache/CacheAbstractTest.cs                     |   70 +-
 .../Cache/CacheAbstractTransactionalTest.cs        |    5 -
 .../Cache/CacheLocalAtomicTest.cs                  |   55 -
 .../Cache/CacheLocalTest.cs                        |   55 -
 .../Cache/Store/CacheStoreTestCodeConfig.cs        |   10 +-
 .../Store/cache-store-session-shared-factory.xml   |    4 +-
 .../Config/Cache/Store/cache-store-session.xml     |    4 +-
 .../Config/full-config.xml                         |    2 +-
 .../native-client-test-cache-parallel-store.xml    |    2 +-
 .../Config/native-client-test-cache-store.xml      |    8 +-
 .../Config/native-client-test-cache.xml            |   12 -
 .../IgniteConfigurationSerializerTest.cs           |    2 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs      |    2 +-
 .../Cache/Configuration/CacheMode.cs               |   16 +-
 .../Cache/Configuration/CacheRebalanceMode.cs      |    5 +-
 .../IgniteConfigurationSection.xsd                 |    1 -
 .../dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs  |    1 +
 .../ignite/cache/websession/WebSessionFilter.java  |    6 -
 .../test/webapp2/META-INF/ignite-webapp-config.xml |   11 -
 .../load/IgniteCacheRandomOperationBenchmark.java  |    4 +-
 .../zk/internal/ZkCommunicationFailureContext.java |    4 -
 361 files changed, 890 insertions(+), 15590 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java
index 805cfa01206..8b9b2fcc731 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java
@@ -235,11 +235,8 @@ public class IndexScan<Row> extends AbstractIndexScan<Row, IndexRow> {
             for (int i = 0; i < parts.length; i++)
                 toReserve.add(top.localPartition(parts[i]));
         }
-        else {
-            assert cctx.isLocal();
-
+        else
             toReserve = Collections.emptyList();
-        }
 
         reserved = new ArrayList<>(toReserve.size());
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java
index 6152959e254..55011b17443 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java
@@ -155,11 +155,8 @@ public class TableScan<Row> implements Iterable<Row>, AutoCloseable {
             for (int i = 0; i < parts.length; i++)
                 toReserve.add(top.localPartition(parts[i]));
         }
-        else {
-            assert cctx.isLocal();
-
+        else
             toReserve = Collections.emptyList();
-        }
 
         reserved = new ArrayList<>(toReserve.size());
 
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
index 455e2675ba3..5543ea138fb 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientAbstractMultiThreadedSelfTest.java
@@ -29,7 +29,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
@@ -44,14 +43,10 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.NotNull;
 import org.junit.Test;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 
 /**
  *
@@ -168,50 +163,29 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
 
         c.setConnectorConfiguration(clientCfg);
 
-        c.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME), cacheConfiguration(PARTITIONED_CACHE_NAME),
-            cacheConfiguration(REPLICATED_CACHE_NAME), cacheConfiguration(PARTITIONED_ASYNC_BACKUP_CACHE_NAME),
-            cacheConfiguration(REPLICATED_ASYNC_CACHE_NAME));
+        CacheConfiguration<?, ?> ccfg1 = defaultCacheConfiguration()
+            .setName(PARTITIONED_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(0);
 
-        return c;
-    }
+        CacheConfiguration<?, ?> ccfg2 = defaultCacheConfiguration()
+            .setName(PARTITIONED_ASYNC_BACKUP_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(1)
+            .setWriteSynchronizationMode(FULL_ASYNC);
 
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     * @throws Exception In case of error.
-     */
-    private CacheConfiguration cacheConfiguration(@NotNull String cacheName) throws Exception {
-        CacheConfiguration cfg = defaultCacheConfiguration();
-
-        cfg.setAffinity(new RendezvousAffinityFunction());
-
-        cfg.setAtomicityMode(TRANSACTIONAL);
-
-        switch (cacheName) {
-            case DEFAULT_CACHE_NAME:
-                cfg.setCacheMode(LOCAL);
-                break;
-            case PARTITIONED_CACHE_NAME:
-                cfg.setCacheMode(PARTITIONED);
-
-                cfg.setBackups(0);
-                break;
-            case PARTITIONED_ASYNC_BACKUP_CACHE_NAME:
-                cfg.setCacheMode(PARTITIONED);
-
-                cfg.setBackups(1);
-                break;
-            default:
-                cfg.setCacheMode(REPLICATED);
-                break;
-        }
+        CacheConfiguration<?, ?> ccfg3 = defaultCacheConfiguration()
+            .setName(REPLICATED_CACHE_NAME)
+            .setCacheMode(REPLICATED);
 
-        cfg.setName(cacheName);
+        CacheConfiguration<?, ?> ccfg4 = defaultCacheConfiguration()
+            .setName(REPLICATED_ASYNC_CACHE_NAME)
+            .setCacheMode(REPLICATED)
+            .setWriteSynchronizationMode(FULL_ASYNC);
 
-        if (!DEFAULT_CACHE_NAME.equals(cacheName) && !cacheName.contains("async"))
-            cfg.setWriteSynchronizationMode(FULL_SYNC);
+        c.setCacheConfiguration(ccfg1, ccfg2, ccfg3, ccfg4);
 
-        return cfg;
+        return c;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientDefaultCacheSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientDefaultCacheSelfTest.java
index 39aa869a056..43922e669e4 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientDefaultCacheSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/ClientDefaultCacheSelfTest.java
@@ -29,9 +29,6 @@ import java.util.Map;
 import java.util.UUID;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
@@ -39,7 +36,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_JETTY_PORT;
 
 /**
@@ -64,9 +60,6 @@ public class ClientDefaultCacheSelfTest extends GridCommonAbstractTest {
     /** Used to sent request charset. */
     private static final String CHARSET = StandardCharsets.UTF_8.name();
 
-    /** Name of node local cache. */
-    private static final String LOCAL_CACHE = "local";
-
     /** JSON to java mapper. */
     private static final ObjectMapper JSON_MAPPER = new ObjectMapper();
 
@@ -99,15 +92,7 @@ public class ClientDefaultCacheSelfTest extends GridCommonAbstractTest {
 
         cfg.setConnectorConfiguration(clientCfg);
 
-        CacheConfiguration cLoc = new CacheConfiguration(DEFAULT_CACHE_NAME);
-
-        cLoc.setName(LOCAL_CACHE);
-
-        cLoc.setCacheMode(CacheMode.LOCAL);
-
-        cLoc.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-
-        cfg.setCacheConfiguration(defaultCacheConfiguration(), cLoc);
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
 
         return cfg;
     }
@@ -177,7 +162,7 @@ public class ClientDefaultCacheSelfTest extends GridCommonAbstractTest {
         String val = "{\"v\":\"my Value\",\"t\":1422559650154}";
 
         // Put to cache JSON format string value.
-        String ret = content(F.asMap("cmd", GridRestCommand.CACHE_PUT.key(), "cacheName", LOCAL_CACHE,
+        String ret = content(F.asMap("cmd", GridRestCommand.CACHE_PUT.key(), "cacheName", DEFAULT_CACHE_NAME,
             "key", "a", "val", URLEncoder.encode(val, CHARSET)));
 
         JsonNode res = jsonResponse(ret);
@@ -185,7 +170,7 @@ public class ClientDefaultCacheSelfTest extends GridCommonAbstractTest {
         assertEquals("Incorrect put response", true, res.asBoolean());
 
         // Escape '\' symbols disappear from response string on transformation to JSON object.
-        ret = content(F.asMap("cmd", GridRestCommand.CACHE_GET.key(), "cacheName", LOCAL_CACHE, "key", "a"));
+        ret = content(F.asMap("cmd", GridRestCommand.CACHE_GET.key(), "cacheName", DEFAULT_CACHE_NAME, "key", "a"));
 
         res = jsonResponse(ret);
 
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
index b5877334613..0d17ee4bc9c 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -28,7 +28,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
@@ -70,13 +69,9 @@ import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.junit.Test;
-
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
@@ -178,47 +173,27 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
 
         c.setCommunicationSpi(spi);
 
-        c.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME), cacheConfiguration(PARTITIONED_CACHE_NAME),
-            cacheConfiguration(REPLICATED_CACHE_NAME), cacheConfiguration(REPLICATED_ASYNC_CACHE_NAME));
-
-        c.setPublicThreadPoolSize(40);
+        CacheConfiguration<?, ?> ccfg1 = defaultCacheConfiguration()
+            .setName(PARTITIONED_CACHE_NAME)
+            .setCacheMode(PARTITIONED)
+            .setBackups(0);
 
-        c.setSystemThreadPoolSize(40);
-
-        return c;
-    }
+        CacheConfiguration<?, ?> ccfg2 = defaultCacheConfiguration()
+            .setName(REPLICATED_CACHE_NAME)
+            .setCacheMode(REPLICATED);
 
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     * @throws Exception In case of error.
-     */
-    private CacheConfiguration cacheConfiguration(@NotNull String cacheName) throws Exception {
-        CacheConfiguration cfg = defaultCacheConfiguration();
-
-        cfg.setAtomicityMode(TRANSACTIONAL);
-
-        switch (cacheName) {
-            case DEFAULT_CACHE_NAME:
-                cfg.setCacheMode(LOCAL);
-                break;
-            case PARTITIONED_CACHE_NAME:
-                cfg.setCacheMode(PARTITIONED);
-
-                cfg.setBackups(0);
-                break;
-            default:
-                cfg.setCacheMode(REPLICATED);
-                break;
-        }
+        CacheConfiguration<?, ?> ccfg3 = defaultCacheConfiguration()
+            .setName(REPLICATED_ASYNC_CACHE_NAME)
+            .setCacheMode(REPLICATED)
+            .setWriteSynchronizationMode(FULL_ASYNC);
 
-        cfg.setName(cacheName);
+        c.setCacheConfiguration(ccfg1, ccfg2, ccfg3);
 
-        cfg.setWriteSynchronizationMode(REPLICATED_ASYNC_CACHE_NAME.equals(cacheName) ? FULL_ASYNC : FULL_SYNC);
+        c.setPublicThreadPoolSize(40);
 
-        cfg.setAffinity(new RendezvousAffinityFunction());
+        c.setSystemThreadPoolSize(40);
 
-        return cfg;
+        return c;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
index 1992643fbb0..c06933c1928 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractSelfTest.java
@@ -70,12 +70,9 @@ import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.junit.Assert;
 import org.junit.Test;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_JETTY_PORT;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
  * Tests for Java client.
@@ -83,7 +80,10 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 @SuppressWarnings("deprecation")
 public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
     /** */
-    private static final String CACHE_NAME = "cache";
+    private static final String REPLICATED_CACHE_NAME = "replicated";
+
+    /** */
+    private static final String PARTITIONED_CACHE_NAME = "partitioned";
 
     /** */
     public static final String HOST = "127.0.0.1";
@@ -153,8 +153,8 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
                 cacheStore.map.clear();
         }
 
-        grid().cache(DEFAULT_CACHE_NAME).clear();
-        grid().cache(CACHE_NAME).clear();
+        grid().cache(PARTITIONED_CACHE_NAME).clear();
+        grid().cache(REPLICATED_CACHE_NAME).clear();
 
         INTERCEPTED_OBJECTS.clear();
     }
@@ -228,8 +228,7 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
 
         cfg.setConnectorConfiguration(clientCfg);
 
-        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME), cacheConfiguration("replicated"),
-            cacheConfiguration("partitioned"), cacheConfiguration(CACHE_NAME));
+        cfg.setCacheConfiguration(cacheConfiguration(REPLICATED_CACHE_NAME), cacheConfiguration(PARTITIONED_CACHE_NAME));
 
         clientCfg.setMessageInterceptor(new ConnectorMessageInterceptor() {
             /** {@inheritDoc} */
@@ -262,10 +261,8 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
     private static CacheConfiguration cacheConfiguration(@NotNull final String cacheName) throws Exception {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
-        cfg.setCacheMode(DEFAULT_CACHE_NAME.equals(cacheName) || CACHE_NAME.equals(cacheName) ? LOCAL : "replicated".equals(cacheName) ?
-            REPLICATED : PARTITIONED);
+        cfg.setCacheMode(REPLICATED_CACHE_NAME.equals(cacheName) ? REPLICATED : PARTITIONED);
         cfg.setName(cacheName);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
 
         cfg.setCacheStoreFactory(new Factory<CacheStore>() {
             @Override public CacheStore create() {
@@ -308,7 +305,7 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
 
         GridClientDataConfiguration cache = new GridClientDataConfiguration();
 
-        cache.setName(CACHE_NAME);
+        cache.setName(PARTITIONED_CACHE_NAME);
 
         cfg.setDataConfigurations(Arrays.asList(nullCache, cache));
 
@@ -320,7 +317,7 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
         cfg.setExecutorService(Executors.newCachedThreadPool(new ThreadFactory() {
             private AtomicInteger cntr = new AtomicInteger();
 
-            @SuppressWarnings("NullableProblems")
+            /** {@inheritDoc} */
             @Override public Thread newThread(Runnable r) {
                 return new Thread(r, "client-worker-thread-" + cntr.getAndIncrement());
             }
@@ -353,7 +350,7 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
     public void testNoAsyncExceptions() throws Exception {
         GridClient client = client();
 
-        GridClientData data = client.data(CACHE_NAME);
+        GridClientData data = client.data(PARTITIONED_CACHE_NAME);
         GridClientCompute compute = client.compute().projection(new GridClientPredicate<GridClientNode>() {
             @Override public boolean apply(GridClientNode e) {
                 return false;
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
deleted file mode 100644
index ba7c176c5a1..00000000000
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
+++ /dev/null
@@ -1,173 +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.internal.jdbc2;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.util.Properties;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.ConnectorConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Test;
-
-import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
-import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- * Test JDBC with several local caches.
- */
-public class JdbcLocalCachesSelfTest extends GridCommonAbstractTest {
-    /** Cache name. */
-    private static final String CACHE_NAME = "cache";
-
-    /** JDBC URL. */
-    private static final String BASE_URL =
-        CFG_URL_PREFIX + "cache=" + CACHE_NAME + "@modules/clients/src/test/config/jdbc-config.xml";
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        CacheConfiguration cache = defaultCacheConfiguration();
-
-        cache.setName(CACHE_NAME);
-        cache.setCacheMode(LOCAL);
-        cache.setWriteSynchronizationMode(FULL_SYNC);
-        cache.setIndexedTypes(
-            String.class, Integer.class
-        );
-
-        cfg.setCacheConfiguration(cache);
-
-        cfg.setConnectorConfiguration(new ConnectorConfiguration());
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(2);
-
-        IgniteCache<Object, Object> cache1 = grid(0).cache(CACHE_NAME);
-
-        assert cache1 != null;
-
-        cache1.put("key1", 1);
-        cache1.put("key2", 2);
-
-        IgniteCache<Object, Object> cache2 = grid(1).cache(CACHE_NAME);
-
-        assert cache2 != null;
-
-        cache2.put("key1", 3);
-        cache2.put("key2", 4);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testCache1() throws Exception {
-        Properties cfg = new Properties();
-
-        cfg.setProperty(PROP_NODE_ID, grid(0).localNode().id().toString());
-
-        Connection conn = null;
-
-        try {
-            conn = DriverManager.getConnection(BASE_URL, cfg);
-
-            ResultSet rs = conn.createStatement().executeQuery("select _val from Integer order by _val");
-
-            int cnt = 0;
-
-            while (rs.next())
-                assertEquals(++cnt, rs.getInt(1));
-
-            assertEquals(2, cnt);
-        }
-        finally {
-            if (conn != null)
-                conn.close();
-        }
-    }
-
-    /**
-     * Verifies that <code>select count(*)</code> behaves correctly in
-     * {@link org.apache.ignite.cache.CacheMode#LOCAL} mode.
-     *
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testCountAll() throws Exception {
-        Properties cfg = new Properties();
-
-        cfg.setProperty(PROP_NODE_ID, grid(0).localNode().id().toString());
-
-        Connection conn = null;
-
-        try {
-            conn = DriverManager.getConnection(BASE_URL, cfg);
-
-            ResultSet rs = conn.createStatement().executeQuery("select count(*) from Integer");
-
-            assertTrue(rs.next());
-
-            assertEquals(2L, rs.getLong(1));
-        }
-        finally {
-            if (conn != null)
-                conn.close();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testCache2() throws Exception {
-        Properties cfg = new Properties();
-
-        cfg.setProperty(PROP_NODE_ID, grid(1).localNode().id().toString());
-
-        Connection conn = null;
-
-        try {
-            conn = DriverManager.getConnection(BASE_URL, cfg);
-
-            ResultSet rs = conn.createStatement().executeQuery("select _val from Integer order by _val");
-
-            int cnt = 0;
-
-            while (rs.next())
-                assertEquals(++cnt + 2, rs.getInt(1));
-
-            assertEquals(2, cnt);
-        }
-        finally {
-            if (conn != null)
-                conn.close();
-        }
-    }
-}
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
index 7d1025f1357..6dbbaa8843a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
@@ -44,9 +44,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.NotNull;
 import org.junit.Test;
 
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
 /**
  * TCP protocol test.
  */
@@ -104,17 +101,11 @@ public class RestBinaryProtocolSelfTest extends GridCommonAbstractTest {
     /**
      * @param cacheName Cache name.
      * @return Cache configuration.
-     * @throws Exception In case of error.
      */
-    private CacheConfiguration cacheConfiguration(@NotNull String cacheName) throws Exception {
-        CacheConfiguration cfg = defaultCacheConfiguration();
-
-        cfg.setCacheMode(LOCAL);
-        cfg.setName(cacheName);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-        cfg.setStatisticsEnabled(true);
-
-        return cfg;
+    private static CacheConfiguration<?, ?> cacheConfiguration(@NotNull String cacheName) {
+        return defaultCacheConfiguration()
+            .setName(cacheName)
+            .setStatisticsEnabled(true);
     }
 
     /**
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestMemcacheProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestMemcacheProtocolSelfTest.java
index 8dec04799c2..12cfa0cb236 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestMemcacheProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestMemcacheProtocolSelfTest.java
@@ -27,9 +27,6 @@ import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
 /**
  * TCP protocol test.
  */
@@ -86,17 +83,11 @@ public class RestMemcacheProtocolSelfTest extends GridCommonAbstractTest {
     /**
      * @param cacheName Cache name.
      * @return Cache configuration.
-     * @throws Exception In case of error.
      */
-    private CacheConfiguration cacheConfiguration(@NotNull String cacheName) throws Exception {
-        CacheConfiguration cfg = defaultCacheConfiguration();
-
-        cfg.setCacheMode(LOCAL);
-        cfg.setName(cacheName);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
-        cfg.setStatisticsEnabled(true);
-
-        return cfg;
+    private static CacheConfiguration<?, ?> cacheConfiguration(@NotNull String cacheName) {
+        return defaultCacheConfiguration()
+            .setName(cacheName)
+            .setStatisticsEnabled(true);
     }
 
     /**
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
index 0a924201d62..5a2d054c2b0 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
@@ -42,14 +42,10 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jetbrains.annotations.NotNull;
 import org.jsr166.ConcurrentLinkedHashMap;
 import org.junit.Test;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.client.GridClientProtocol.TCP;
 
 /**
@@ -57,7 +53,10 @@ import static org.apache.ignite.internal.client.GridClientProtocol.TCP;
  */
 public class TaskCommandHandlerSelfTest extends GridCommonAbstractTest {
     /** */
-    private static final String CACHE_NAME = "cache";
+    private static final String REPLICATED_CACHE_NAME = "replicated";
+
+    /** */
+    private static final String PARTITIONED_CACHE_NAME = "partitioned";
 
     /** */
     public static final String HOST = "127.0.0.1";
@@ -107,24 +106,15 @@ public class TaskCommandHandlerSelfTest extends GridCommonAbstractTest {
 
         cfg.setConnectorConfiguration(clientCfg);
 
-        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME), cacheConfiguration("replicated"),
-            cacheConfiguration("partitioned"), cacheConfiguration(CACHE_NAME));
-
-        return cfg;
-    }
+        CacheConfiguration<?, ?> cfg1 = defaultCacheConfiguration()
+            .setName(REPLICATED_CACHE_NAME)
+            .setCacheMode(REPLICATED);
 
-    /**
-     * @param cacheName Cache name.
-     * @return Cache configuration.
-     * @throws Exception In case of error.
-     */
-    private CacheConfiguration cacheConfiguration(@NotNull String cacheName) throws Exception {
-        CacheConfiguration cfg = defaultCacheConfiguration();
+        CacheConfiguration<?, ?> cfg2 = defaultCacheConfiguration()
+            .setName(PARTITIONED_CACHE_NAME)
+            .setCacheMode(PARTITIONED);
 
-        cfg.setCacheMode(DEFAULT_CACHE_NAME.equals(cacheName) || CACHE_NAME.equals(cacheName) ? LOCAL : "replicated".equals(cacheName) ?
-            REPLICATED : PARTITIONED);
-        cfg.setName(cacheName);
-        cfg.setWriteSynchronizationMode(FULL_SYNC);
+        cfg.setCacheConfiguration(cfg1, cfg2);
 
         return cfg;
     }
@@ -139,12 +129,12 @@ public class TaskCommandHandlerSelfTest extends GridCommonAbstractTest {
 
         GridClientDataConfiguration cache = new GridClientDataConfiguration();
 
-        cache.setName(CACHE_NAME);
+        cache.setName(PARTITIONED_CACHE_NAME);
 
         cfg.setDataConfigurations(Arrays.asList(nullCache, cache));
 
         cfg.setProtocol(TCP);
-        cfg.setServers(Arrays.asList("localhost:" + BINARY_PORT));
+        cfg.setServers(Collections.singletonList("localhost:" + BINARY_PORT));
 
         return cfg;
     }
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcLocalCachesSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcLocalCachesSelfTest.java
deleted file mode 100644
index 4810d102952..00000000000
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcLocalCachesSelfTest.java
+++ /dev/null
@@ -1,141 +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.jdbc;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.util.Properties;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.ConnectorConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Test;
-
-import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- * Test JDBC with several local caches.
- */
-public class JdbcLocalCachesSelfTest extends GridCommonAbstractTest {
-    /** Cache name. */
-    private static final String CACHE_NAME = "cache";
-
-    /** URL. */
-    private static final String URL = "jdbc:ignite://127.0.0.1/" + CACHE_NAME;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        CacheConfiguration cache = defaultCacheConfiguration();
-
-        cache.setName(CACHE_NAME);
-        cache.setCacheMode(LOCAL);
-        cache.setWriteSynchronizationMode(FULL_SYNC);
-        cache.setIndexedTypes(
-            String.class, Integer.class
-        );
-
-        cfg.setCacheConfiguration(cache);
-
-        cfg.setConnectorConfiguration(new ConnectorConfiguration());
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(2);
-
-        IgniteCache<Object, Object> cache1 = grid(0).cache(CACHE_NAME);
-
-        assert cache1 != null;
-
-        cache1.put("key1", 1);
-        cache1.put("key2", 2);
-
-        IgniteCache<Object, Object> cache2 = grid(1).cache(CACHE_NAME);
-
-        assert cache2 != null;
-
-        cache2.put("key1", 3);
-        cache2.put("key2", 4);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testCache1() throws Exception {
-        Properties cfg = new Properties();
-
-        cfg.setProperty(PROP_NODE_ID, grid(0).localNode().id().toString());
-
-        Connection conn = null;
-
-        try {
-            conn = DriverManager.getConnection(URL, cfg);
-
-            ResultSet rs = conn.createStatement().executeQuery("select _val from Integer order by _val");
-
-            int cnt = 0;
-
-            while (rs.next())
-                assertEquals(++cnt, rs.getInt(1));
-
-            assertEquals(2, cnt);
-        }
-        finally {
-            if (conn != null)
-                conn.close();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testCache2() throws Exception {
-        Properties cfg = new Properties();
-
-        cfg.setProperty(PROP_NODE_ID, grid(1).localNode().id().toString());
-
-        Connection conn = null;
-
-        try {
-            conn = DriverManager.getConnection(URL, cfg);
-
-            ResultSet rs = conn.createStatement().executeQuery("select _val from Integer order by _val");
-
-            int cnt = 0;
-
-            while (rs.next())
-                assertEquals(++cnt + 2, rs.getInt(1));
-
-            assertEquals(2, cnt);
-        }
-        finally {
-            if (conn != null)
-                conn.close();
-        }
-    }
-}
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index e1c3ed8e27f..47a3429c000 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -29,7 +29,6 @@ import org.apache.ignite.jdbc.JdbcComplexQuerySelfTest;
 import org.apache.ignite.jdbc.JdbcConnectionSelfTest;
 import org.apache.ignite.jdbc.JdbcDefaultNoOpCacheTest;
 import org.apache.ignite.jdbc.JdbcEmptyCacheSelfTest;
-import org.apache.ignite.jdbc.JdbcLocalCachesSelfTest;
 import org.apache.ignite.jdbc.JdbcMetadataSelfTest;
 import org.apache.ignite.jdbc.JdbcNoDefaultCacheTest;
 import org.apache.ignite.jdbc.JdbcPojoLegacyQuerySelfTest;
@@ -120,7 +119,6 @@ import org.junit.runners.Suite;
     JdbcComplexQuerySelfTest.class,
     JdbcMetadataSelfTest.class,
     JdbcEmptyCacheSelfTest.class,
-    JdbcLocalCachesSelfTest.class,
     JdbcNoDefaultCacheTest.class,
     JdbcDefaultNoOpCacheTest.class,
     JdbcPojoQuerySelfTest.class,
@@ -139,7 +137,6 @@ import org.junit.runners.Suite;
     JdbcDistributedJoinsQueryTest.class,
     org.apache.ignite.internal.jdbc2.JdbcMetadataSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcEmptyCacheSelfTest.class,
-    org.apache.ignite.internal.jdbc2.JdbcLocalCachesSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcConnectionWithoutCacheNameTest.class,
     org.apache.ignite.internal.jdbc2.JdbcMergeStatementSelfTest.class,
     org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerMergeStatementSelfTest.class,
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinCacheToJdbcDataTypesCoverageTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinCacheToJdbcDataTypesCoverageTest.java
index b250d71abb9..02cbc0906bc 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinCacheToJdbcDataTypesCoverageTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinCacheToJdbcDataTypesCoverageTest.java
@@ -46,7 +46,6 @@ import java.util.function.Function;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
@@ -62,7 +61,6 @@ import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
-
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 
 /**
@@ -418,8 +416,7 @@ public class JdbcThinCacheToJdbcDataTypesCoverageTest extends GridCacheDataTypes
 
         Class<?> dataType = originalValItem.getClass();
 
-        IgniteEx ignite =
-            (cacheMode == CacheMode.LOCAL || writeSyncMode == CacheWriteSynchronizationMode.PRIMARY_SYNC) ?
+        IgniteEx ignite = writeSyncMode == CacheWriteSynchronizationMode.PRIMARY_SYNC ?
                 grid(0) :
                 grid(new Random().nextInt(NODES_CNT));
 
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinJdbcToCacheDataTypesCoverageTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinJdbcToCacheDataTypesCoverageTest.java
index bb43e3a1819..9de4259ceae 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinJdbcToCacheDataTypesCoverageTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinJdbcToCacheDataTypesCoverageTest.java
@@ -23,7 +23,6 @@ import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.UUID;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -33,7 +32,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
-
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 
 /**
@@ -55,7 +53,6 @@ public class JdbcThinJdbcToCacheDataTypesCoverageTest extends SqlDataTypesCovera
     private Statement stmt;
 
     /** @inheritDoc */
-    @SuppressWarnings("RedundantMethodOverride")
     @Before
     @Override public void init() throws Exception {
         super.init();
@@ -131,8 +128,7 @@ public class JdbcThinJdbcToCacheDataTypesCoverageTest extends SqlDataTypesCovera
     @Override protected void checkBasicSqlOperations(SqlDataType dataType, Object... valsToCheck) throws Exception {
         assert valsToCheck.length > 0;
 
-        IgniteEx ignite =
-            (cacheMode == CacheMode.LOCAL || writeSyncMode == CacheWriteSynchronizationMode.PRIMARY_SYNC) ?
+        IgniteEx ignite = writeSyncMode == CacheWriteSynchronizationMode.PRIMARY_SYNC ?
                 grid(0) :
                 grid(new Random().nextInt(NODES_CNT));
 
@@ -163,8 +159,7 @@ public class JdbcThinJdbcToCacheDataTypesCoverageTest extends SqlDataTypesCovera
             " val " + dataType + ")" +
             " WITH " + "\"template=" + templateName + ",cache_name=" + cacheName + ",wrap_value=false\"");
 
-        if (cacheMode != CacheMode.LOCAL)
-            stmt.execute("CREATE INDEX " + idxName + " ON " + tblName + "(id, val)");
+        stmt.execute("CREATE INDEX " + idxName + " ON " + tblName + "(id, val)");
 
         for (Object valToCheck : valsToCheck) {
             Object sqlStrVal = valToCheck instanceof SqlStrConvertedValHolder ?
diff --git a/modules/clients/src/test/resources/spring-cache.xml b/modules/clients/src/test/resources/spring-cache.xml
index 86946f6791d..61bcd0aa667 100644
--- a/modules/clients/src/test/resources/spring-cache.xml
+++ b/modules/clients/src/test/resources/spring-cache.xml
@@ -108,17 +108,6 @@
                     <!-- Set synchronous rebalancing (default is asynchronous). -->
                     <property name="rebalanceMode" value="SYNC"/>
                 </bean>
-
-                <!--
-                    Local cache example configuration.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <!-- Cache name is 'local'. -->
-                    <property name="name" value="local"/>
-
-                    <!-- LOCAL cache mode. -->
-                    <property name="cacheMode" value="LOCAL"/>
-                </bean>
             </list>
         </property>
 
diff --git a/modules/clients/src/test/resources/spring-server-node.xml b/modules/clients/src/test/resources/spring-server-node.xml
index 2c27b2f2cdf..4d0d8d65475 100644
--- a/modules/clients/src/test/resources/spring-server-node.xml
+++ b/modules/clients/src/test/resources/spring-server-node.xml
@@ -86,17 +86,6 @@
                 CacheConfiguration interface can be configured here.
             -->
             <list>
-                <!--
-                    Local cache example configuration.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <!-- Cache name is null. -->
-                    <!--<property name="name"><null/></property>-->
-
-                    <!-- LOCAL cache mode. -->
-                    <property name="cacheMode" value="LOCAL"/>
-                </bean>
-
                 <!--
                     Partitioned cache example configuration.
                 -->
@@ -200,21 +189,6 @@
                     </property>
                 </bean>
 
-                <!--
-                    Local cache with store.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="local.store"/>
-
-                    <property name="cacheMode" value="LOCAL"/>
-
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-
-                    <property name="store">
-                        <bean class="org.apache.ignite.internal.client.HashMapStore"/>
-                    </property>
-                </bean>
-
                 <!--
                     Replicated cache example configuration.
                 -->
diff --git a/modules/clients/src/test/resources/spring-server-ssl-node.xml b/modules/clients/src/test/resources/spring-server-ssl-node.xml
index d8e16ff230f..8589a3c6ffe 100644
--- a/modules/clients/src/test/resources/spring-server-ssl-node.xml
+++ b/modules/clients/src/test/resources/spring-server-ssl-node.xml
@@ -87,18 +87,6 @@
                 CacheConfiguration interface can be configured here.
             -->
             <list>
-                <!--
-                    Local cache example configuration.
-                -->
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <!-- Cache name is null. -->
-                    <!--<property name="name"><null/></property>-->
-
-                    <!-- LOCAL cache mode. -->
-                    <property name="cacheMode" value="LOCAL"/>
-
-                </bean>
-
                 <!--
                     Partitioned cache example configuration.
                 -->
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/cache/LocalCacheTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/cache/LocalCacheTest.java
deleted file mode 100644
index e100b0031d2..00000000000
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/cache/LocalCacheTest.java
+++ /dev/null
@@ -1,136 +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.compatibility.cache;
-
-import java.io.File;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.compatibility.persistence.IgnitePersistenceCompatibilityAbstractTest;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.DataRegionConfiguration;
-import org.apache.ignite.configuration.DataStorageConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgnitionEx;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.jetbrains.annotations.Nullable;
-import org.junit.Test;
-
-/**
- * Tests that local cache with persistence enabled can be started on a new version of AI.
- * This test should be removed along with GridCacheProcessor#LocalAffinityFunction.
- */
-public class LocalCacheTest extends IgnitePersistenceCompatibilityAbstractTest {
-    /** Consistent id. */
-    private static final String CONSISTENT_ID = "test-local-cache-id";
-
-    /** Ignite version. */
-    private static final String IGNITE_VERSION = "2.8.0";
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testMigrationLocalCacheWithPersistenceEnabled() throws Exception {
-        try {
-            U.delete(new File(U.defaultWorkDirectory()));
-
-            startGrid(1, IGNITE_VERSION, new ConfigurationClosure(CONSISTENT_ID), new ActivateClosure());
-
-            stopAllGrids();
-
-            Ignite ig0 = IgnitionEx.start(prepareConfig(getConfiguration(), CONSISTENT_ID));
-
-            ig0.close();
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * Updates the given ignite configuration and specifies a local cache with persistence enabled.
-     *
-     * @param cfg Ignite configuration to be updated.
-     * @param consistentId Consistent id.
-     * @return Updated configuration.
-     */
-    private static IgniteConfiguration prepareConfig(IgniteConfiguration cfg, @Nullable String consistentId) {
-        cfg.setLocalHost("127.0.0.1");
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-        disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        cfg.setPeerClassLoadingEnabled(false);
-
-        DataStorageConfiguration memCfg = new DataStorageConfiguration()
-            .setDefaultDataRegionConfiguration(
-                new DataRegionConfiguration()
-                    .setPersistenceEnabled(true)
-                    .setInitialSize(10L * 1024 * 1024)
-                    .setMaxSize(10L * 1024 * 1024))
-            .setPageSize(4096);
-
-        cfg.setDataStorageConfiguration(memCfg);
-
-        if (consistentId != null) {
-            cfg.setIgniteInstanceName(consistentId);
-            cfg.setConsistentId(consistentId);
-        }
-
-        CacheConfiguration<Object, Object> locCacheCfg = new CacheConfiguration<>("test-local-cache");
-        locCacheCfg.setCacheMode(CacheMode.LOCAL);
-        cfg.setCacheConfiguration(locCacheCfg);
-
-        return cfg;
-    }
-
-    /** */
-    private static class ConfigurationClosure implements IgniteInClosure<IgniteConfiguration> {
-        /** Consistent id. */
-        private final String consistentId;
-
-        /**
-         * Creates a new instance of Configuration closure.
-         *
-         * @param consistentId Consistent id.
-         */
-        public ConfigurationClosure(String consistentId) {
-            this.consistentId = consistentId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void apply(IgniteConfiguration cfg) {
-            prepareConfig(cfg, consistentId);
-        }
-    }
-
-    /**
-     * Post-startup close that activates the grid.
-     */
-    private static class ActivateClosure implements IgniteInClosure<Ignite> {
-        /** {@inheritDoc} */
-        @Override public void apply(Ignite ignite) {
-            ignite.cluster().active(true);
-        }
-    }
-}
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/cache/package-info.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/cache/package-info.java
deleted file mode 100644
index dcf7d53cfee..00000000000
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/cache/package-info.java
+++ /dev/null
@@ -1,22 +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.
- */
-
-/**
- * Contains compatibility tests related to caches with persistence enabled.
- */
-
-package org.apache.ignite.compatibility.cache;
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
index 7af12353ffe..c155d1979cf 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.compatibility.testsuites;
 
-import org.apache.ignite.compatibility.cache.LocalCacheTest;
 import org.apache.ignite.compatibility.clients.JavaThinCompatibilityTest;
 import org.apache.ignite.compatibility.clients.JdbcThinCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.FoldersReuseCompatibilityTest;
@@ -41,7 +40,6 @@ import org.junit.runners.Suite;
     FoldersReuseCompatibilityTest.class,
     MigratingToWalV2SerializerWithCompactionTest.class,
     MetaStorageCompatibilityTest.class,
-    LocalCacheTest.class,
     MoveBinaryMetadataCompatibility.class,
     JdbcThinCompatibilityTest.class,
     JavaThinCompatibilityTest.class
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
index 5a82b25379e..392bb615dfe 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.cache;
 
+import java.util.stream.Stream;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.jetbrains.annotations.Nullable;
@@ -26,29 +27,13 @@ import org.jetbrains.annotations.Nullable;
  * and cannot be changed after cache has started.
  */
 public enum CacheMode {
-    /**
-     * Specifies local-only cache behaviour. In this mode caches residing on
-     * different grid nodes will not know about each other.
-     * <p>
-     * Other than distribution, {@code local} caches still have all
-     * the caching features, such as eviction, expiration, swapping,
-     * querying, etc... This mode is very useful when caching read-only data
-     * or data that automatically expires at a certain interval and
-     * then automatically reloaded from persistence store.
-     *
-     * @deprecated Use {@link #REPLICATED} or {@link #PARTITIONED} modes instead.
-     * Please, be aware this API will be removed in the next releases.
-     */
-    @Deprecated
-    LOCAL,
-
     /**
      * Specifies fully replicated cache behavior. In this mode all the keys are distributed
      * to all participating nodes. User still has affinity control
      * over subset of nodes for any given key via {@link AffinityFunction}
      * configuration.
      */
-    REPLICATED,
+    REPLICATED((byte)1),
 
     /**
      * Specifies partitioned cache behaviour. In this mode the overall
@@ -62,18 +47,56 @@ public enum CacheMode {
      * can configure the size of near cache via {@link NearCacheConfiguration#getNearEvictionPolicyFactory()}
      * configuration property.
      */
-    PARTITIONED;
+    PARTITIONED((byte)2);
+
+    /** Cached enumerated values by their codes. */
+    private static final CacheMode[] BY_CODE;
+
+    static {
+        int max = Stream.of(values())
+            .mapToInt(e -> e.code)
+            .max()
+            .orElseThrow(RuntimeException::new);
+
+        BY_CODE = new CacheMode[max + 1];
+
+        for (CacheMode e : values()) {
+            BY_CODE[e.code] = e;
+        }
+    }
+
+    /** Cache mode code. */
+    private final byte code;
+
+    /**
+     * @param code Cache mode code.
+     */
+    CacheMode(byte code) {
+        this.code = code;
+    }
 
-    /** Enumerated values. */
-    private static final CacheMode[] VALS = values();
+    /**
+     * @return Cache mode code.
+     */
+    public byte code() {
+        return code;
+    }
 
     /**
-     * Efficiently gets enumerated value from its ordinal.
+     * Efficiently gets enumerated value from its code.
      *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
+     * @param code Code.
+     * @return Enumerated value or {@code null} if an out of range.
+     */
+    @Nullable public static CacheMode fromCode(int code) {
+        return code >= 0 && code < BY_CODE.length ? BY_CODE[code] : null;
+    }
+
+    /**
+     * @param mode Cache mode.
+     * @return Cache mode code.
      */
-    @Nullable public static CacheMode fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    public static byte toCode(@Nullable CacheMode mode) {
+        return mode == null ? -1 : mode.code;
     }
 }
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 eee1ba9ebe2..108021c6eea 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
@@ -43,19 +43,16 @@ public enum CachePeekMode {
 
     /**
      * Peek into near cache only (don't peek into partitioned cache).
-     * In case of {@link CacheMode#LOCAL} cache, behaves as {@link #ALL} mode.
      */
     NEAR,
 
     /**
      * Peek value from primary copy of partitioned cache only (skip near cache).
-     * In case of {@link CacheMode#LOCAL} cache, behaves as {@link #ALL} mode.
      */
     PRIMARY,
 
     /**
      * Peek value from backup copies of partitioned cache only (skip near cache).
-     * In case of {@link CacheMode#LOCAL} cache, behaves as {@link #ALL} mode.
      */
     BACKUP,
 
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheRebalanceMode.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheRebalanceMode.java
index 57148e86b34..9901922d382 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheRebalanceMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheRebalanceMode.java
@@ -31,8 +31,7 @@ import org.jetbrains.annotations.Nullable;
  * current node is primary or back up.
  * <p>
  * Note that rebalance mode only makes sense for {@link CacheMode#REPLICATED} and {@link CacheMode#PARTITIONED}
- * caches. Caches with {@link CacheMode#LOCAL} mode are local by definition and therefore cannot rebalance
- * any values from neighboring nodes.
+ * caches.
  */
 public enum CacheRebalanceMode {
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 6a67355c4b4..3ffebc67b6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2242,7 +2242,7 @@ public class IgnitionEx {
                     Map<UUID, Map<Integer, Set<Integer>>> proposedSuppliers = new HashMap<>();
 
                     for (CacheGroupContext grpCtx : grid.context().cache().cacheGroups()) {
-                        if (grpCtx.isLocal() || grpCtx.systemCache())
+                        if (grpCtx.systemCache())
                             continue;
 
                         if (grpCtx.config().getCacheMode() == PARTITIONED && grpCtx.config().getBackups() == 0) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientCacheMode.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientCacheMode.java
index b4c0022e564..7e7f3e27cc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientCacheMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientCacheMode.java
@@ -21,9 +21,6 @@ package org.apache.ignite.internal.client;
  * Cache type on remote node.
  */
 public enum GridClientCacheMode {
-    /** Local cache. */
-    LOCAL,
-
     /** Replicated cache. */
     REPLICATED,
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
index 90c4633fd84..d7e30463ed3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
@@ -67,7 +67,6 @@ import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpir
 import org.apache.ignite.internal.util.MutableSingletonList;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.internal.client.thin.ProtocolVersionFeature.EXPIRY_POLICY;
 import static org.apache.ignite.internal.client.thin.ProtocolVersionFeature.QUERY_ENTITY_PRECISION_AND_SCALE;
 import static org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicy.convertDuration;
@@ -265,7 +264,7 @@ public final class ClientUtils {
             };
 
             itemWriter.accept(CfgItem.NAME, w -> w.writeString(cfg.getName()));
-            itemWriter.accept(CfgItem.CACHE_MODE, w -> w.writeInt(cfg.getCacheMode().ordinal()));
+            itemWriter.accept(CfgItem.CACHE_MODE, w -> w.writeInt(CacheMode.toCode(cfg.getCacheMode())));
             itemWriter.accept(CfgItem.ATOMICITY_MODE, w -> w.writeInt(cfg.getAtomicityMode().ordinal()));
             itemWriter.accept(CfgItem.BACKUPS, w -> w.writeInt(cfg.getBackups()));
             itemWriter.accept(CfgItem.WRITE_SYNC_MODE, w -> w.writeInt(cfg.getWriteSynchronizationMode().ordinal()));
@@ -388,7 +387,7 @@ public final class ClientUtils {
             return new ClientCacheConfiguration().setName("TBD") // cache name is to be assigned later
                 .setAtomicityMode(CacheAtomicityMode.fromOrdinal(reader.readInt()))
                 .setBackups(reader.readInt())
-                .setCacheMode(CacheMode.fromOrdinal(reader.readInt()))
+                .setCacheMode(CacheMode.fromCode(reader.readInt()))
                 .setCopyOnRead(reader.readBoolean())
                 .setDataRegionName(reader.readString())
                 .setEagerTtl(reader.readBoolean())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
index f1d0bd5b90d..82e7bb04ca2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.client.ClientAtomicConfiguration;
@@ -362,7 +363,7 @@ public class TcpIgniteClient implements IgniteClient {
                 if (cfg != null) {
                     out.out().writeBoolean(true);
                     out.out().writeInt(cfg.getAtomicSequenceReserveSize());
-                    out.out().writeByte((byte)cfg.getCacheMode().ordinal());
+                    out.out().writeByte(CacheMode.toCode(cfg.getCacheMode()));
                     out.out().writeInt(cfg.getBackups());
                     writeString(cfg.getGroupName(), out.out());
                 }
@@ -390,7 +391,7 @@ public class TcpIgniteClient implements IgniteClient {
             if (cfg != null) {
                 out.out().writeBoolean(true);
                 out.out().writeByte((byte)cfg.getAtomicityMode().ordinal());
-                out.out().writeByte((byte)cfg.getCacheMode().ordinal());
+                out.out().writeByte(CacheMode.toCode(cfg.getCacheMode()));
                 out.out().writeInt(cfg.getBackups());
                 writeString(cfg.getGroupName(), out.out());
                 out.out().writeBoolean(cfg.isColocated());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java
index 6ea1bd2e494..d3a1ce3cb45 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java
@@ -68,9 +68,6 @@ class JdbcBatchUpdateTask implements IgniteCallable<int[]> {
     /** Fetch size. */
     private final int fetchSize;
 
-    /** Local execution flag. */
-    private final boolean loc;
-
     /** Local query flag. */
     private final boolean locQry;
 
@@ -87,14 +84,13 @@ class JdbcBatchUpdateTask implements IgniteCallable<int[]> {
      * @param sql SQL query. {@code null} in case of statement batching.
      * @param sqlBatch Batch of SQL statements. {@code null} in case of parameter batching.
      * @param batchArgs Batch of SQL parameters. {@code null} in case of statement batching.
-     * @param loc Local execution flag.
      * @param fetchSize Fetch size.
      * @param locQry Local query flag.
      * @param collocatedQry Collocated query flag.
      * @param distributedJoins Distributed joins flag.
      */
     public JdbcBatchUpdateTask(Ignite ignite, String cacheName, String schemaName, String sql,
-        List<String> sqlBatch, List<List<Object>> batchArgs, boolean loc, int fetchSize,
+        List<String> sqlBatch, List<List<Object>> batchArgs, int fetchSize,
         boolean locQry, boolean collocatedQry, boolean distributedJoins) {
         this.ignite = ignite;
         this.cacheName = cacheName;
@@ -103,7 +99,6 @@ class JdbcBatchUpdateTask implements IgniteCallable<int[]> {
         this.sqlBatch = sqlBatch;
         this.batchArgs = batchArgs;
         this.fetchSize = fetchSize;
-        this.loc = loc;
         this.locQry = locQry;
         this.collocatedQry = collocatedQry;
         this.distributedJoins = distributedJoins;
@@ -119,7 +114,7 @@ class JdbcBatchUpdateTask implements IgniteCallable<int[]> {
         boolean start = ignite.configuration().isClientMode();
 
         if (cache == null && cacheName == null)
-            cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start, !loc && locQry);
+            cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start);
 
         if (cache == null) {
             if (cacheName == null) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
index fce046dd752..b26fca548dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
@@ -150,7 +150,7 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTaskResult> {
             boolean start = ignite.configuration().isClientMode();
 
             if (cache == null && cacheName == null)
-                cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start, !loc && locQry);
+                cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start);
 
             if (cache == null) {
                 if (cacheName == null)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
index 20554d7a801..591fb0a32cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
@@ -463,7 +463,7 @@ public class JdbcStatement implements Statement {
             throw new SQLException("Failed to query Ignite: DML operations are supported in versions 1.8.0 and newer");
 
         JdbcBatchUpdateTask task = new JdbcBatchUpdateTask(loc ? ignite : null, conn.cacheName(),
-            conn.schemaName(), command, batch, batchArgs, loc, getFetchSize(), conn.isLocalQuery(),
+            conn.schemaName(), command, batch, batchArgs, getFetchSize(), conn.isLocalQuery(),
             conn.isCollocatedQuery(), conn.isDistributedJoins());
 
         try {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 0bc08e29e5e..e5628392f2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -55,12 +55,10 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD;
 import static org.apache.ignite.IgniteSystemProperties.getFloat;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 
@@ -142,9 +140,6 @@ public class GridAffinityAssignmentCache {
     /** */
     private final GridKernalContext ctx;
 
-    /** */
-    private final boolean locCache;
-
     /** Node stop flag. */
     private volatile IgniteCheckedException stopErr;
 
@@ -163,15 +158,13 @@ public class GridAffinityAssignmentCache {
      * @param aff Affinity function.
      * @param nodeFilter Node filter.
      * @param backups Number of backups.
-     * @param locCache Local cache flag.
      */
     private GridAffinityAssignmentCache(GridKernalContext ctx,
         String cacheOrGrpName,
         int grpId,
         AffinityFunction aff,
         IgnitePredicate<ClusterNode> nodeFilter,
-        int backups,
-        boolean locCache
+        int backups
     ) {
         assert ctx != null;
         assert aff != null;
@@ -184,7 +177,6 @@ public class GridAffinityAssignmentCache {
         this.cacheOrGrpName = cacheOrGrpName;
         this.grpId = grpId;
         this.backups = backups;
-        this.locCache = locCache;
 
         log = ctx.log(GridAffinityAssignmentCache.class);
 
@@ -209,8 +201,7 @@ public class GridAffinityAssignmentCache {
             CU.cacheGroupId(ccfg),
             aff,
             ccfg.getNodeFilter(),
-            ccfg.getBackups(),
-            ccfg.getCacheMode() == LOCAL);
+            ccfg.getBackups());
     }
 
     /**
@@ -351,15 +342,9 @@ public class GridAffinityAssignmentCache {
             return prevAssignment;
 
         // Resolve nodes snapshot for specified topology version.
-        List<ClusterNode> sorted;
+        List<ClusterNode> sorted = new ArrayList<>(discoCache.cacheGroupAffinityNodes(groupId()));
 
-        if (!locCache) {
-            sorted = new ArrayList<>(discoCache.cacheGroupAffinityNodes(groupId()));
-
-            sorted.sort(NodeOrderComparator.getInstance());
-        }
-        else
-            sorted = Collections.singletonList(ctx.discovery().localNode());
+        sorted.sort(NodeOrderComparator.getInstance());
 
         boolean hasBaseline = false;
         boolean changedBaseline = false;
@@ -463,9 +448,6 @@ public class GridAffinityAssignmentCache {
             baselineAssignment = null;
         }
 
-        if (locCache)
-            initialize(topVer, assignment.assignment());
-
         return assignment;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
index 0972878e035..caddc85d9b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
@@ -70,8 +70,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridClosureCallMode.BROADCAST;
@@ -515,9 +513,6 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
             return new GridFinishedFuture<>((AffinityInfo)null);
         }
 
-        if (desc.cacheConfiguration().getCacheMode() == LOCAL)
-            return new GridFinishedFuture<>(new IgniteCheckedException("Failed to map keys for LOCAL cache: " + cacheName));
-
         AffinityFuture fut0 = new AffinityFuture(cacheName, topVer, cacheNodes);
 
         IgniteInternalFuture<AffinityInfo> old = affMap.putIfAbsent(key, fut0);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/LocalAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/LocalAffinityFunction.java
deleted file mode 100644
index 963de8cd02b..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/LocalAffinityFunction.java
+++ /dev/null
@@ -1,78 +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.internal.processors.affinity;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.AffinityFunctionContext;
-import org.apache.ignite.cluster.ClusterNode;
-
-/**
- * Affinity function for {@link org.apache.ignite.cache.CacheMode#LOCAL} caches.
- */
-public class LocalAffinityFunction implements AffinityFunction {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public List<List<ClusterNode>> assignPartitions(AffinityFunctionContext affCtx) {
-        ClusterNode locNode = null;
-
-        for (ClusterNode n : affCtx.currentTopologySnapshot()) {
-            if (n.isLocal()) {
-                locNode = n;
-
-                break;
-            }
-        }
-
-        if (locNode == null)
-            throw new IgniteException("Local node is not included into affinity nodes for 'LOCAL' cache");
-
-        List<List<ClusterNode>> res = new ArrayList<>(partitions());
-
-        for (int part = 0; part < partitions(); part++)
-            res.add(Collections.singletonList(locNode));
-
-        return Collections.unmodifiableList(res);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key) {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeNode(UUID nodeId) {
-        // No-op.
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 639268da437..f6f4a954be2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -78,8 +78,6 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
@@ -495,11 +493,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 CacheGroupContext grp = cctx.cache().cacheGroup(grpDesc.groupId());
 
                 assert grp != null : grpDesc.groupId();
-                assert !grp.affinityNode() || grp.isLocal() : grp.cacheOrGroupName();
-
-                // Skip for local caches.
-                if (grp.isLocal())
-                    continue;
+                assert !grp.affinityNode() : grp.cacheOrGroupName();
 
                 CacheGroupHolder grpHolder = grpHolders.get(grp.groupId());
 
@@ -615,13 +609,11 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         }
 
         for (DynamicCacheDescriptor desc : startDescs) {
-            if (desc.cacheConfiguration().getCacheMode() != LOCAL) {
-                CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
+            CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
 
-                assert grp != null;
+            assert grp != null;
 
-                grp.topology().onExchangeDone(null, grp.affinity().cachedAffinity(topVer), true);
-            }
+            grp.topology().onExchangeDone(null, grp.affinity().cachedAffinity(topVer), true);
         }
 
         cctx.cache().initCacheProxies(topVer, null);
@@ -1086,17 +1078,15 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         Set<Integer> stoppedGrps = null;
 
         for (ExchangeActions.CacheGroupActionData data : exchActions.cacheGroupsToStop()) {
-            if (data.descriptor().config().getCacheMode() != LOCAL) {
-                CacheGroupHolder cacheGrp = grpHolders.remove(data.descriptor().groupId());
+            CacheGroupHolder cacheGrp = grpHolders.remove(data.descriptor().groupId());
 
-                if (cacheGrp != null) {
-                    if (stoppedGrps == null)
-                        stoppedGrps = new HashSet<>();
+            if (cacheGrp != null) {
+                if (stoppedGrps == null)
+                    stoppedGrps = new HashSet<>();
 
-                    stoppedGrps.add(cacheGrp.groupId());
+                stoppedGrps.add(cacheGrp.groupId());
 
-                    cctx.io().removeHandler(true, cacheGrp.groupId(), GridDhtAffinityAssignmentResponse.class);
-                }
+                cctx.io().removeHandler(true, cacheGrp.groupId(), GridDhtAffinityAssignmentResponse.class);
             }
         }
 
@@ -1284,9 +1274,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
      * @param c Cache closure.
      */
     private void forAllRegisteredCacheGroups(IgniteInClosureX<CacheGroupDescriptor> c) {
-        Collection<CacheGroupDescriptor> affinityCaches = cachesRegistry.allGroups().values().stream()
-            .filter(desc -> desc.config().getCacheMode() != LOCAL)
-            .collect(Collectors.toList());
+        Collection<CacheGroupDescriptor> affinityCaches = new ArrayList<>(cachesRegistry.allGroups().values());
 
         try {
             U.doInParallel(cctx.kernalContext().pools().getSystemExecutorService(), affinityCaches, t -> {
@@ -1329,9 +1317,6 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         throws IgniteCheckedException {
         assert grpDesc != null && grpDesc.groupId() != 0 : grpDesc;
 
-        if (grpDesc.config().getCacheMode() == LOCAL)
-            return;
-
         int grpId = grpDesc.groupId();
 
         CacheGroupHolder grpHolder = grpHolders.get(grpId);
@@ -2707,8 +2692,6 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
         CacheGroupAffNodeHolder(CacheGroupContext grp, @Nullable GridAffinityAssignmentCache initAff) {
             super(grp.rebalanceEnabled(), grp.affinity(), initAff);
 
-            assert !grp.isLocal() : grp;
-
             this.grp = grp;
         }
 
@@ -2767,7 +2750,6 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             CacheConfiguration<?, ?> ccfg = grpDesc.config();
 
             assert ccfg != null : grpDesc;
-            assert ccfg.getCacheMode() != LOCAL : ccfg.getName();
 
             assert !cctx.discovery().cacheGroupAffinityNodes(grpDesc.groupId(),
                 topVer).contains(cctx.localNode()) : grpDesc.cacheOrGroupName();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index 9bd805100c2..e26a8b93ef4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -71,10 +71,8 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CacheTopologyValidatorProvider;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_MISSED;
@@ -701,13 +699,6 @@ public class CacheGroupContext {
         return locStartVer;
     }
 
-    /**
-     * @return {@code True} if cache is local.
-     */
-    public boolean isLocal() {
-        return ccfg.getCacheMode() == LOCAL;
-    }
-
     /**
      * @return {@code True} if cache is local.
      */
@@ -878,18 +869,14 @@ public class CacheGroupContext {
     private void initializeIO() throws IgniteCheckedException {
         assert !recoveryMode.get() : "Couldn't initialize I/O handlers, recovery mode is on for group " + this;
 
-        if (ccfg.getCacheMode() != LOCAL) {
-            if (!ctx.kernalContext().clientNode()) {
-                ctx.io().addCacheGroupHandler(groupId(), GridDhtAffinityAssignmentRequest.class,
-                    (IgniteBiInClosure<UUID, GridDhtAffinityAssignmentRequest>)this::processAffinityAssignmentRequest);
-            }
+        if (!ctx.kernalContext().clientNode()) {
+            ctx.io().addCacheGroupHandler(groupId(), GridDhtAffinityAssignmentRequest.class,
+                (IgniteBiInClosure<UUID, GridDhtAffinityAssignmentRequest>)this::processAffinityAssignmentRequest);
+        }
 
-            preldr = new GridDhtPreloader(this);
+        preldr = new GridDhtPreloader(this);
 
-            preldr.start();
-        }
-        else
-            preldr = new GridCachePreloaderAdapter(this);
+        preldr.start();
     }
 
     /**
@@ -944,7 +931,6 @@ public class CacheGroupContext {
     public void addCacheWithContinuousQuery(GridCacheContext cctx) {
         assert sharedGroup() : cacheOrGroupName();
         assert cctx.group() == this : cctx.name();
-        assert !cctx.isLocal() : cctx.name();
 
         List<GridCacheContext> contQryCaches = this.contQryCaches;
 
@@ -962,7 +948,6 @@ public class CacheGroupContext {
     public void removeCacheWithContinuousQuery(GridCacheContext cctx) {
         assert sharedGroup() : cacheOrGroupName();
         assert cctx.group() == this : cctx.name();
-        assert !cctx.isLocal() : cctx.name();
         assert listenerLock.isWriteLockedByCurrentThread();
 
         List<GridCacheContext> contQryCaches = this.contQryCaches;
@@ -998,12 +983,10 @@ public class CacheGroupContext {
         int part,
         long cntr,
         AffinityTopologyVersion topVer,
-        boolean primary) {
+        boolean primary
+    ) {
         assert sharedGroup();
 
-        if (isLocal())
-            return;
-
         List<GridCacheContext> contQryCaches;
 
         listenerLock.readLock().lock();
@@ -1065,11 +1048,9 @@ public class CacheGroupContext {
 
         aff = affCache == null ? GridAffinityAssignmentCache.create(ctx.kernalContext(), ccfg.getAffinity(), ccfg) : affCache;
 
-        if (ccfg.getCacheMode() != LOCAL) {
-            top = ctx.kernalContext().resource().resolve(new GridDhtPartitionTopologyImpl(ctx, this));
+        top = ctx.kernalContext().resource().resolve(new GridDhtPartitionTopologyImpl(ctx, this));
 
-            metrics.onTopologyInitialized();
-        }
+        metrics.onTopologyInitialized();
 
         try {
             offheapMgr = ctx.kernalContext().resource().resolve(persistenceEnabled
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 8522fbc2036..be49398e4cf 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
@@ -1288,9 +1288,6 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @return Valid ot not.
      */
     private boolean isValidForOperation(boolean read) {
-        if (cctx.isLocal())
-            return true;
-
         try {
             GridDhtTopologyFuture fut = cctx.shared().exchange().lastFinishedFuture();
 
@@ -1359,43 +1356,34 @@ public class CacheMetricsImpl implements CacheMetrics {
                 sizeLong = cache.localSizeLong(null);
             }
 
-            if (cctx.isLocal()) {
-                if (cache != null) {
-                    offHeapPrimaryEntriesCnt = offHeapEntriesCnt;
+            IntSet primaries = ImmutableIntSet.wrap(cctx.affinity().primaryPartitions(cctx.localNodeId(), topVer));
+            IntSet backups = ImmutableIntSet.wrap(cctx.affinity().backupPartitions(cctx.localNodeId(), topVer));
 
-                    heapEntriesCnt = cache.sizeLong();
-                }
-            }
-            else {
-                IntSet primaries = ImmutableIntSet.wrap(cctx.affinity().primaryPartitions(cctx.localNodeId(), topVer));
-                IntSet backups = ImmutableIntSet.wrap(cctx.affinity().backupPartitions(cctx.localNodeId(), topVer));
+            if (cctx.isNear() && cache != null)
+                heapEntriesCnt = cache.nearSize();
 
-                if (cctx.isNear() && cache != null)
-                    heapEntriesCnt = cache.nearSize();
+            for (GridDhtLocalPartition part : cctx.topology().currentLocalPartitions()) {
+                // Partitions count.
+                GridDhtPartitionState partState = part.state();
 
-                for (GridDhtLocalPartition part : cctx.topology().currentLocalPartitions()) {
-                    // Partitions count.
-                    GridDhtPartitionState partState = part.state();
+                if (partState == GridDhtPartitionState.OWNING)
+                    owningPartCnt++;
 
-                    if (partState == GridDhtPartitionState.OWNING)
-                        owningPartCnt++;
+                if (partState == GridDhtPartitionState.MOVING)
+                    movingPartCnt++;
 
-                    if (partState == GridDhtPartitionState.MOVING)
-                        movingPartCnt++;
+                // Offheap entries count
+                if (cache == null)
+                    continue;
 
-                    // Offheap entries count
-                    if (cache == null)
-                        continue;
+                long cacheSize = part.dataStore().cacheSize(cctx.cacheId());
 
-                    long cacheSize = part.dataStore().cacheSize(cctx.cacheId());
+                if (primaries.contains(part.id()))
+                    offHeapPrimaryEntriesCnt += cacheSize;
+                else if (backups.contains(part.id()))
+                    offHeapBackupEntriesCnt += cacheSize;
 
-                    if (primaries.contains(part.id()))
-                        offHeapPrimaryEntriesCnt += cacheSize;
-                    else if (backups.contains(part.id()))
-                        offHeapBackupEntriesCnt += cacheSize;
-
-                    heapEntriesCnt += part.publicSize(cctx.cacheId());
-                }
+                heapEntriesCnt += part.publicSize(cctx.cacheId());
             }
         }
         catch (Exception e) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 29f268478d3..2944c2f7ceb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -85,9 +85,7 @@ import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.systemview.view.CacheGroupView;
 import org.apache.ignite.spi.systemview.view.CacheView;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
@@ -402,123 +400,121 @@ public class ClusterCachesInfo {
         CU.checkAttributeMismatch(log, rmtAttr.groupName(), rmt, "groupName", "Cache group name",
             locAttr.groupName(), rmtAttr.groupName(), true);
 
-        if (rmtAttr.cacheMode() != LOCAL) {
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "interceptor", "Cache Interceptor",
-                locAttr.interceptorClassName(), rmtAttr.interceptorClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "interceptor", "Cache Interceptor",
+            locAttr.interceptorClassName(), rmtAttr.interceptorClassName(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "atomicityMode",
-                "Cache atomicity mode", locAttr.atomicityMode(), rmtAttr.atomicityMode(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "atomicityMode",
+            "Cache atomicity mode", locAttr.atomicityMode(), rmtAttr.atomicityMode(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
-                "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
+            "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "topologyValidator",
-                "Cache topology validator", locAttr.topologyValidatorClassName(), rmtAttr.topologyValidatorClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "topologyValidator",
+            "Cache topology validator", locAttr.topologyValidatorClassName(), rmtAttr.topologyValidatorClassName(), true);
 
-            ClusterNode rmtNode = ctx.discovery().node(rmt);
+        ClusterNode rmtNode = ctx.discovery().node(rmt);
 
-            if (CU.affinityNode(ctx.discovery().localNode(), locInfo.cacheData().config().getNodeFilter())
-                && rmtNode != null && CU.affinityNode(rmtNode, rmtData.cacheConfiguration().getNodeFilter())) {
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
-                    locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
-            }
+        if (CU.affinityNode(ctx.discovery().localNode(), locInfo.cacheData().config().getNodeFilter())
+            && rmtNode != null && CU.affinityNode(rmtNode, rmtData.cacheConfiguration().getNodeFilter())) {
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
+                locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
+        }
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheAffinity", "Cache affinity",
-                locAttr.cacheAffinityClassName(), rmtAttr.cacheAffinityClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheAffinity", "Cache affinity",
+            locAttr.cacheAffinityClassName(), rmtAttr.cacheAffinityClassName(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheAffinityMapper",
-                "Cache affinity mapper", locAttr.cacheAffinityMapperClassName(),
-                rmtAttr.cacheAffinityMapperClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheAffinityMapper",
+            "Cache affinity mapper", locAttr.cacheAffinityMapperClassName(),
+            rmtAttr.cacheAffinityMapperClassName(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityPartitionsCount",
-                "Affinity partitions count", locAttr.affinityPartitionsCount(),
-                rmtAttr.affinityPartitionsCount(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityPartitionsCount",
+            "Affinity partitions count", locAttr.affinityPartitionsCount(),
+            rmtAttr.affinityPartitionsCount(), true);
 
-            CU.validateKeyConfigiration(rmtAttr.groupName(), rmtAttr.cacheName(), rmt, rmtAttr.configuration().getKeyConfiguration(),
-                locAttr.configuration().getKeyConfiguration(), log, true);
+        CU.validateKeyConfigiration(rmtAttr.groupName(), rmtAttr.cacheName(), rmt, rmtAttr.configuration().getKeyConfiguration(),
+            locAttr.configuration().getKeyConfiguration(), log, true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionFilter", "Eviction filter",
-                locAttr.evictionFilterClassName(), rmtAttr.evictionFilterClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionFilter", "Eviction filter",
+            locAttr.evictionFilterClassName(), rmtAttr.evictionFilterClassName(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicy", "Eviction policy",
-                locAttr.evictionPolicyClassName(), rmtAttr.evictionPolicyClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicy", "Eviction policy",
+            locAttr.evictionPolicyClassName(), rmtAttr.evictionPolicyClassName(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicyFactory", "Eviction policy factory",
-                locAttr.evictionPolicyFactoryClassName(), rmtAttr.evictionPolicyFactoryClassName(), true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicyFactory", "Eviction policy factory",
+            locAttr.evictionPolicyFactoryClassName(), rmtAttr.evictionPolicyFactoryClassName(), true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "transactionManagerLookup",
-                "Transaction manager lookup", locAttr.transactionManagerLookupClassName(),
-                rmtAttr.transactionManagerLookupClassName(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "transactionManagerLookup",
+            "Transaction manager lookup", locAttr.transactionManagerLookupClassName(),
+            rmtAttr.transactionManagerLookupClassName(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "defaultLockTimeout",
-                "Default lock timeout", locAttr.defaultLockTimeout(), rmtAttr.defaultLockTimeout(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "defaultLockTimeout",
+            "Default lock timeout", locAttr.defaultLockTimeout(), rmtAttr.defaultLockTimeout(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "preloadBatchSize",
-                "Preload batch size", locAttr.rebalanceBatchSize(), rmtAttr.rebalanceBatchSize(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "preloadBatchSize",
+            "Preload batch size", locAttr.rebalanceBatchSize(), rmtAttr.rebalanceBatchSize(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceDelay",
-                "Rebalance delay", locAttr.rebalanceDelay(), rmtAttr.rebalanceDelay(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceDelay",
+            "Rebalance delay", locAttr.rebalanceDelay(), rmtAttr.rebalanceDelay(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceBatchesPrefetchCount",
-                "Rebalance batches prefetch count", locAttr.rebalanceBatchesPrefetchCount(),
-                rmtAttr.rebalanceBatchesPrefetchCount(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceBatchesPrefetchCount",
+            "Rebalance batches prefetch count", locAttr.rebalanceBatchesPrefetchCount(),
+            rmtAttr.rebalanceBatchesPrefetchCount(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceOrder",
-                "Rebalance order", locAttr.rebalanceOrder(), rmtAttr.rebalanceOrder(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceOrder",
+            "Rebalance order", locAttr.rebalanceOrder(), rmtAttr.rebalanceOrder(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceThrottle",
-                "Rebalance throttle", locAttr.rebalanceThrottle(), rmtAttr.rebalanceThrottle(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceThrottle",
+            "Rebalance throttle", locAttr.rebalanceThrottle(), rmtAttr.rebalanceThrottle(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceTimeout",
-                "Rebalance timeout", locAttr.rebalanceTimeout(), rmtAttr.rebalanceTimeout(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "rebalanceTimeout",
+            "Rebalance timeout", locAttr.rebalanceTimeout(), rmtAttr.rebalanceTimeout(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeSynchronizationMode",
-                "Write synchronization mode", locAttr.writeSynchronization(), rmtAttr.writeSynchronization(),
-                true);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeSynchronizationMode",
+            "Write synchronization mode", locAttr.writeSynchronization(), rmtAttr.writeSynchronization(),
+            true);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindBatchSize",
-                "Write behind batch size", locAttr.writeBehindBatchSize(), rmtAttr.writeBehindBatchSize(),
-                false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindBatchSize",
+            "Write behind batch size", locAttr.writeBehindBatchSize(), rmtAttr.writeBehindBatchSize(),
+            false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindCoalescing",
-                "Write behind coalescing", locAttr.writeBehindCoalescing(), rmtAttr.writeBehindCoalescing(),
-                false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindCoalescing",
+            "Write behind coalescing", locAttr.writeBehindCoalescing(), rmtAttr.writeBehindCoalescing(),
+            false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindEnabled",
-                "Write behind enabled", locAttr.writeBehindEnabled(), rmtAttr.writeBehindEnabled(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindEnabled",
+            "Write behind enabled", locAttr.writeBehindEnabled(), rmtAttr.writeBehindEnabled(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindFlushFrequency",
-                "Write behind flush frequency", locAttr.writeBehindFlushFrequency(),
-                rmtAttr.writeBehindFlushFrequency(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindFlushFrequency",
+            "Write behind flush frequency", locAttr.writeBehindFlushFrequency(),
+            rmtAttr.writeBehindFlushFrequency(), false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindFlushSize",
-                "Write behind flush size", locAttr.writeBehindFlushSize(), rmtAttr.writeBehindFlushSize(),
-                false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindFlushSize",
+            "Write behind flush size", locAttr.writeBehindFlushSize(), rmtAttr.writeBehindFlushSize(),
+            false);
 
-            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindFlushThreadCount",
-                "Write behind flush thread count", locAttr.writeBehindFlushThreadCount(),
-                rmtAttr.writeBehindFlushThreadCount(), false);
+        CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "writeBehindFlushThreadCount",
+            "Write behind flush thread count", locAttr.writeBehindFlushThreadCount(),
+            rmtAttr.writeBehindFlushThreadCount(), false);
 
-            if (locAttr.cacheMode() == PARTITIONED) {
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicy",
-                    "Near eviction policy", locAttr.nearEvictionPolicyClassName(),
-                    rmtAttr.nearEvictionPolicyClassName(), false);
+        if (locAttr.cacheMode() == PARTITIONED) {
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicy",
+                "Near eviction policy", locAttr.nearEvictionPolicyClassName(),
+                rmtAttr.nearEvictionPolicyClassName(), false);
 
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicyFactory",
-                    "Near eviction policy factory", locAttr.nearEvictionPolicyFactoryClassName(),
-                    rmtAttr.nearEvictionPolicyFactoryClassName(), false);
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicyFactory",
+                "Near eviction policy factory", locAttr.nearEvictionPolicyFactoryClassName(),
+                rmtAttr.nearEvictionPolicyFactoryClassName(), false);
 
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityIncludeNeighbors",
-                    "Affinity include neighbors", locAttr.affinityIncludeNeighbors(),
-                    rmtAttr.affinityIncludeNeighbors(), true);
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityIncludeNeighbors",
+                "Affinity include neighbors", locAttr.affinityIncludeNeighbors(),
+                rmtAttr.affinityIncludeNeighbors(), true);
 
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityKeyBackups",
-                    "Affinity key backups", locAttr.affinityKeyBackups(),
-                    rmtAttr.affinityKeyBackups(), true);
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityKeyBackups",
+                "Affinity key backups", locAttr.affinityKeyBackups(),
+                rmtAttr.affinityKeyBackups(), true);
 
-                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "qryParallelism",
-                    "Query parallelism", locAttr.qryParallelism(), rmtAttr.qryParallelism(), true);
-            }
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "qryParallelism",
+                "Query parallelism", locAttr.qryParallelism(), rmtAttr.qryParallelism(), true);
         }
 
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "isEncryptionEnabled",
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
index 862e0f17067..bf945a19a8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
@@ -253,9 +253,6 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
                 throw new UnsupportedOperationException("Read-repair is incompatible with caches that use readThrough.");
             }
 
-            if (context().isLocal())
-                throw new UnsupportedOperationException("Read-repair is incompatible with local caches.");
-
             if (context().config().getBackups() == 0) {
                 throw new UnsupportedOperationException("Read-repair is suitable only in case " +
                     "at least 1 backup configured for cache.");
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 30a63949621..1598e0e2f19 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
@@ -770,11 +770,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         final boolean keepBinary = ctx.keepBinary();
 
-        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());
@@ -831,103 +826,86 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         CacheObject cacheVal = null;
 
-        if (!ctx.isLocal()) {
-            AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
+        AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 
-            int part = ctx.affinity().partition(cacheKey);
+        int part = ctx.affinity().partition(cacheKey);
 
-            boolean nearKey;
+        boolean nearKey;
+
+        if (!(modes.near && modes.primary && modes.backup)) {
+            boolean keyPrimary = ctx.affinity().primaryByPartition(ctx.localNode(), part, topVer);
+
+            if (keyPrimary) {
+                if (!modes.primary)
+                    return null;
 
-            if (!(modes.near && modes.primary && modes.backup)) {
-                boolean keyPrimary = ctx.affinity().primaryByPartition(ctx.localNode(), part, topVer);
+                nearKey = false;
+            }
+            else {
+                boolean keyBackup = ctx.affinity().partitionBelongs(ctx.localNode(), part, topVer);
 
-                if (keyPrimary) {
-                    if (!modes.primary)
+                if (keyBackup) {
+                    if (!modes.backup)
                         return null;
 
                     nearKey = false;
                 }
                 else {
-                    boolean keyBackup = ctx.affinity().partitionBelongs(ctx.localNode(), part, topVer);
-
-                    if (keyBackup) {
-                        if (!modes.backup)
-                            return null;
-
-                        nearKey = false;
-                    }
-                    else {
-                        if (!modes.near)
-                            return null;
-
-                        nearKey = true;
+                    if (!modes.near)
+                        return null;
 
-                        // Swap and offheap are disabled for near cache.
-                        modes.offheap = false;
-                    }
-                }
-            }
-            else {
-                nearKey = !ctx.affinity().partitionBelongs(ctx.localNode(), part, topVer);
+                    nearKey = true;
 
-                if (nearKey) {
                     // Swap and offheap are disabled for near cache.
                     modes.offheap = false;
                 }
             }
+        }
+        else {
+            nearKey = !ctx.affinity().partitionBelongs(ctx.localNode(), part, topVer);
 
-            if (nearKey && !ctx.isNear())
-                return null;
-
-            GridCacheEntryEx e;
-            GridCacheContext ctx0;
-
-            while (true) {
-                if (nearKey)
-                    e = peekEx(key);
-                else {
-                    ctx0 = ctx.isNear() ? ctx.near().dht().context() : ctx;
-                    e = modes.offheap ? ctx0.cache().entryEx(key) : ctx0.cache().peekEx(key);
-                }
+            if (nearKey) {
+                // Swap and offheap are disabled for near cache.
+                modes.offheap = false;
+            }
+        }
 
-                if (e != null) {
-                    ctx.shared().database().checkpointReadLock();
+        if (nearKey && !ctx.isNear())
+            return null;
 
-                    try {
-                        cacheVal = ctx.mvccEnabled()
-                            ? e.mvccPeek(modes.heap && !modes.offheap)
-                            : e.peek(modes.heap, modes.offheap, topVer, null);
-                    }
-                    catch (GridCacheEntryRemovedException ignore) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry during 'peek': " + key);
+        GridCacheEntryEx e;
+        GridCacheContext ctx0;
 
-                        continue;
-                    }
-                    finally {
-                        e.touch();
+        while (true) {
+            if (nearKey)
+                e = peekEx(key);
+            else {
+                ctx0 = ctx.isNear() ? ctx.near().dht().context() : ctx;
+                e = modes.offheap ? ctx0.cache().entryEx(key) : ctx0.cache().peekEx(key);
+            }
 
-                        ctx.shared().database().checkpointReadUnlock();
-                    }
-                }
+            if (e != null) {
+                ctx.shared().database().checkpointReadLock();
 
-                break;
-            }
-        }
-        else {
-            while (true) {
                 try {
-                    cacheVal = localCachePeek0(cacheKey, modes.heap, modes.offheap);
-
-                    break;
+                    cacheVal = ctx.mvccEnabled()
+                        ? e.mvccPeek(modes.heap && !modes.offheap)
+                        : e.peek(modes.heap, modes.offheap, topVer, null);
                 }
                 catch (GridCacheEntryRemovedException ignore) {
                     if (log.isDebugEnabled())
                         log.debug("Got removed entry during 'peek': " + key);
 
-                    // continue
+                    continue;
+                }
+                finally {
+                    e.touch();
+
+                    ctx.shared().database().checkpointReadUnlock();
                 }
             }
+
+            break;
         }
 
         Object val = ctx.unwrapBinaryIfNeeded(cacheVal, ctx.keepBinary(), false, null);
@@ -935,35 +913,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return (V)val;
     }
 
-    /**
-     * @param key Key.
-     * @param heap Read heap flag.
-     * @param offheap Read offheap flag.
-     * @return Value.
-     * @throws GridCacheEntryRemovedException If entry removed.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable private CacheObject localCachePeek0(KeyCacheObject key,
-        boolean heap,
-        boolean offheap)
-        throws GridCacheEntryRemovedException, IgniteCheckedException {
-        assert ctx.isLocal();
-        assert heap || offheap;
-
-        GridCacheEntryEx e = offheap ? entryEx(key) : peekEx(key);
-
-        if (e != null) {
-            try {
-                return e.peek(heap, offheap, AffinityTopologyVersion.NONE, null);
-            }
-            finally {
-                e.touch();
-            }
-        }
-
-        return null;
-    }
-
     /**
      * Undeploys and removes all entries for class loader.
      *
@@ -1206,16 +1155,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         ctx.shared().cache().checkReadOnlyState("clear", ctx.config());
 
-        if (isLocal()) {
-            if (keys == null)
-                clearLocally(true, false, false);
-            else
-                clearLocallyAll(keys, true, false, false);
-        }
-        else {
-            executeClearTask(keys, false).get();
-            executeClearTask(keys, true).get();
-        }
+        executeClearTask(keys, false).get();
+        executeClearTask(keys, true).get();
     }
 
     /**
@@ -1228,27 +1169,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         ctx.shared().cache().checkReadOnlyState("clear", ctx.config());
 
-        if (isLocal())
-            return clearLocallyAsync(keys);
-        else
-            return executeClearTask(keys, false).chainCompose(fut -> executeClearTask(keys, true));
-    }
-
-    /**
-     * @param keys Keys to clear.
-     * @return Clear future.
-     */
-    private IgniteInternalFuture<?> clearLocallyAsync(@Nullable final Set<? extends K> keys) {
-        return ctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-            @Override public Object call() {
-                if (keys == null)
-                    clearLocally(true, false, false);
-                else
-                    clearLocallyAll(keys, true, false, false);
-
-                return null;
-            }
-        }, false);
+        return executeClearTask(keys, false).chainCompose(fut -> executeClearTask(keys, true));
     }
 
     /**
@@ -1364,9 +1285,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Collection<Integer> lostPartitions() {
-        if (isLocal())
-            return Collections.emptyList();
-
         return ctx.topology().lostPartitions();
     }
 
@@ -3240,8 +3158,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void removeAll() throws IgniteCheckedException {
-        assert ctx.isLocal();
-
         // We do batch and recreate cursor because removing using a single cursor
         // will cause it to reinitialize on each merged page.
         List<K> keys = new ArrayList<>(Math.min(REMOVE_ALL_KEYS_BATCH, size()));
@@ -4038,25 +3954,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public int size(CachePeekMode[] peekModes) throws IgniteCheckedException {
-        if (isLocal())
-            return localSize(peekModes);
-
         return sizeAsync(peekModes).get();
     }
 
     /** {@inheritDoc} */
     @Override public long sizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException {
-        if (isLocal())
-            return localSizeLong(peekModes);
-
         return sizeLongAsync(peekModes).get();
     }
 
     /** {@inheritDoc} */
     @Override public long sizeLong(int partition, CachePeekMode[] peekModes) throws IgniteCheckedException {
-        if (isLocal())
-            return localSizeLong(partition, peekModes);
-
         return sizeLongAsync(partition, peekModes).get();
     }
 
@@ -5434,28 +5341,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void preloadPartition(int part) throws IgniteCheckedException {
-        if (isLocal())
-            ctx.offheap().preloadPartition(part);
-        else
-            executePreloadTask(part).get();
+        executePreloadTask(part).get();
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> preloadPartitionAsync(int part) throws IgniteCheckedException {
-        if (isLocal()) {
-            return ctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
-                @Override public void run() {
-                    try {
-                        ctx.offheap().preloadPartition(part);
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IgniteException(e);
-                    }
-                }
-            });
-        }
-        else
-            return executePreloadTask(part);
+        return executePreloadTask(part);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index 855923ffdc9..8b551040998 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -64,13 +64,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
         aff = cctx.group().affinity();
     }
 
-    /** {@inheritDoc} */
-    @Override protected void onKernalStart0() throws IgniteCheckedException {
-        if (cctx.isLocal())
-            // No discovery event needed for local affinity.
-            aff.calculate(LOC_CACHE_TOP_VER, null, null);
-    }
-
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel, boolean destroy) {
         aff = null;
@@ -95,8 +88,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Affinity ready future.
      */
     public IgniteInternalFuture<AffinityTopologyVersion> affinityReadyFuture(AffinityTopologyVersion topVer) {
-        assert !cctx.isLocal();
-
         IgniteInternalFuture<AffinityTopologyVersion> fut = aff.readyFuture(topVer);
 
         return fut != null ? fut : new GridFinishedFuture<>(aff.lastVersion());
@@ -110,8 +101,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Affinity ready future or {@code null}.
      */
     @Nullable public IgniteInternalFuture<AffinityTopologyVersion> affinityReadyFuturex(AffinityTopologyVersion topVer) {
-        assert !cctx.isLocal();
-
         return aff.readyFuture(topVer);
     }
 
@@ -120,9 +109,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Affinity assignments.
      */
     public List<List<ClusterNode>> assignments(AffinityTopologyVersion topVer) {
-        if (cctx.isLocal())
-            topVer = LOC_CACHE_TOP_VER;
-
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
@@ -135,8 +121,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Assignment.
      */
     public List<List<ClusterNode>> idealAssignment() {
-        assert !cctx.isLocal();
-
         return aff.idealAssignmentRaw();
     }
 
@@ -216,9 +200,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Affinity nodes.
      */
     public List<ClusterNode> nodesByPartition(int part, AffinityTopologyVersion topVer) {
-        if (cctx.isLocal())
-            topVer = LOC_CACHE_TOP_VER;
-
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
@@ -244,9 +225,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Affinity assignment.
      */
     public AffinityAssignment assignment(AffinityTopologyVersion topVer, AffinityTopologyVersion lastAffChangedTopVer) {
-        if (cctx.isLocal())
-            topVer = lastAffChangedTopVer = LOC_CACHE_TOP_VER;
-
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
@@ -376,9 +354,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Partitions for which given node is primary.
      */
     public Set<Integer> primaryPartitions(UUID nodeId, AffinityTopologyVersion topVer) {
-        if (cctx.isLocal())
-            topVer = LOC_CACHE_TOP_VER;
-
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
@@ -393,9 +368,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return Partitions for which given node is backup.
      */
     public Set<Integer> backupPartitions(UUID nodeId, AffinityTopologyVersion topVer) {
-        if (cctx.isLocal())
-            topVer = LOC_CACHE_TOP_VER;
-
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
@@ -423,8 +395,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      * @return {@code True} if primary changed or required affinity version not found in history.
      */
     public boolean primaryChanged(int part, AffinityTopologyVersion startVer, AffinityTopologyVersion endVer) {
-        assert !cctx.isLocal() : cctx.name();
-
         GridAffinityAssignmentCache aff0 = aff;
 
         if (aff0 == null)
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 abb299f3177..466f6cbfafd 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
@@ -30,8 +30,6 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_ATOMICITY_MODE;
 import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_MODE;
 
@@ -120,7 +118,7 @@ public class GridCacheAttributes implements Serializable {
      * @return {@code True} if near cache is enabled.
      */
     public boolean nearCacheEnabled() {
-        return cacheMode() != LOCAL && ccfg.getNearConfiguration() != null;
+        return ccfg.getNearConfiguration() != null;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 670b5a6f08a..89f15fe57ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -78,7 +78,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCach
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrManager;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
-import org.apache.ignite.internal.processors.cache.local.GridLocalCache;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
@@ -115,7 +114,6 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.security.SecurityException;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISABLE_TRIGGERING_CACHE_INTERCEPTOR_ON_CONFLICT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_READ_LOAD_BALANCING;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -644,13 +642,6 @@ public class GridCacheContext<K, V> implements Externalizable {
             (cache.isNear() && cache.configuration().getAtomicityMode() == ATOMIC);
     }
 
-    /**
-     * @return Local cache.
-     */
-    public GridLocalCache<K, V> local() {
-        return (GridLocalCache<K, V>)cache;
-    }
-
     /**
      * @return {@code True} if cache is DHT.
      */
@@ -679,13 +670,6 @@ public class GridCacheContext<K, V> implements Externalizable {
         return cache != null && cache.isNear();
     }
 
-    /**
-     * @return {@code True} if cache is local.
-     */
-    public boolean isLocal() {
-        return cache != null && cache.isLocal();
-    }
-
     /**
      * @return {@code True} if cache is replicated cache.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index 083e7dfb5a3..1d6dab1753c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -223,19 +223,6 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
                 }
             });
         }
-
-        // Unwind immediately for local and replicate caches.
-        // We go through preloader for proper synchronization.
-        if (ctx.isLocal()) {
-            ctx.preloader().pause();
-
-            try {
-                ctx.group().unwindUndeploys();
-            }
-            finally {
-                ctx.preloader().resume();
-            }
-        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 3b380d5306e..431ba34b017 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -21,9 +21,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
 import javax.cache.Cache;
-import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.eviction.EvictableEntry;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -41,7 +39,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
 import org.apache.ignite.internal.processors.dr.GridDrType;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
-import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
@@ -569,45 +566,6 @@ public interface GridCacheEntryEx {
         boolean transformOp
     ) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
-    /**
-     * Update method for local cache in atomic mode.
-     *
-     * @param ver Cache version.
-     * @param op Operation.
-     * @param writeObj Value. Type depends on operation.
-     * @param invokeArgs Optional arguments for EntryProcessor.
-     * @param writeThrough Write through flag.
-     * @param readThrough Read through flag.
-     * @param retval Return value flag.
-     * @param expiryPlc Expiry policy..
-     * @param evt Event flag.
-     * @param metrics Metrics update flag.
-     * @param filter Optional filter to check.
-     * @param intercept If {@code true} then calls cache interceptor.
-     * @param taskName Task name.
-     * @param transformOp {@code True} if transform operation caused update.
-     * @return Tuple containing success flag, old value and result for invoke operation.
-     * @throws IgniteCheckedException If update failed.
-     * @throws GridCacheEntryRemovedException If entry is obsolete.
-     */
-    public GridTuple3<Boolean, Object, EntryProcessorResult<Object>> innerUpdateLocal(
-        GridCacheVersion ver,
-        GridCacheOperation op,
-        @Nullable Object writeObj,
-        @Nullable Object[] invokeArgs,
-        boolean writeThrough,
-        boolean readThrough,
-        boolean retval,
-        boolean keepBinary,
-        @Nullable ExpiryPolicy expiryPlc,
-        boolean evt,
-        boolean metrics,
-        @Nullable CacheEntryPredicate[] filter,
-        boolean intercept,
-        String taskName,
-        boolean transformOp
-    ) throws IgniteCheckedException, GridCacheEntryRemovedException;
-
     /**
      * Marks entry as obsolete and, if possible or required, removes it
      * from swap storage.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index c10fd4736b7..d939635cc27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -32,7 +32,6 @@ import java.util.function.Supplier;
 import javax.cache.Cache;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -98,7 +97,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -109,7 +107,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.IgniteSystemProperties.getLong;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
@@ -1887,361 +1884,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             return new GridCacheUpdateTxResult(false, logPtr);
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public GridTuple3<Boolean, Object, EntryProcessorResult<Object>> innerUpdateLocal(
-        GridCacheVersion ver,
-        GridCacheOperation op,
-        @Nullable Object writeObj,
-        @Nullable Object[] invokeArgs,
-        boolean writeThrough,
-        boolean readThrough,
-        boolean retval,
-        boolean keepBinary,
-        @Nullable ExpiryPolicy expiryPlc,
-        boolean evt,
-        boolean metrics,
-        @Nullable CacheEntryPredicate[] filter,
-        boolean intercept,
-        String taskName,
-        boolean transformOp
-    ) throws IgniteCheckedException, GridCacheEntryRemovedException {
-        assert cctx.isLocal() && cctx.atomic();
-
-        CacheObject old;
-
-        boolean res = true;
-
-        IgniteBiTuple<Boolean, ?> interceptorRes = null;
-
-        EntryProcessorResult<Object> invokeRes = null;
-
-        lockListenerReadLock();
-        lockEntry();
-
-        try {
-            checkObsolete();
-
-            boolean internal = isInternal() || !context().userCache();
-
-            Map<UUID, CacheContinuousQueryListener> lsnrCol =
-                cctx.continuousQueries().updateListeners(internal, false);
-
-            boolean needVal = retval ||
-                intercept ||
-                op == GridCacheOperation.TRANSFORM ||
-                !F.isEmpty(filter) ||
-                lsnrCol != null;
-
-            // Load and remove from swap if it is new.
-            if (isNew())
-                unswap(null, false);
-
-            old = val;
-
-            if (expireTimeExtras() > 0 && expireTimeExtras() < U.currentTimeMillis()) {
-                if (onExpired(val, null)) {
-                    assert !deletedUnlocked();
-
-                    update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true);
-
-                    old = null;
-                }
-            }
-
-            boolean readFromStore = false;
-
-            Object old0 = null;
-
-            if (readThrough && needVal && old == null &&
-                (cctx.readThrough() && (op == GridCacheOperation.TRANSFORM || cctx.loadPreviousValue()))) {
-                old0 = readThrough(null, key, false, taskName);
-
-                old = cctx.toCacheObject(old0);
-
-                long ttl = CU.TTL_ETERNAL;
-                long expireTime = CU.EXPIRE_TIME_ETERNAL;
-
-                if (expiryPlc != null && old != null) {
-                    ttl = CU.toTtl(expiryPlc.getExpiryForCreation());
-
-                    if (ttl == CU.TTL_ZERO) {
-                        ttl = CU.TTL_MINIMUM;
-                        expireTime = CU.expireTimeInPast();
-                    }
-                    else if (ttl == CU.TTL_NOT_CHANGED)
-                        ttl = CU.TTL_ETERNAL;
-                    else
-                        expireTime = CU.toExpireTime(ttl);
-                }
-
-                // Detach value before index update.
-                old = cctx.kernalContext().cacheObjects().prepareForCache(old, cctx);
-
-                if (old != null)
-                    storeValue(old, expireTime, ver);
-                else
-                    removeValue();
-
-                update(old, expireTime, ttl, ver, true);
-            }
-
-            // Apply metrics.
-            if (metrics && cctx.statisticsEnabled() && needVal)
-                cctx.cache().metrics0().onRead(old != null);
-
-            // Check filter inside of synchronization.
-            if (!F.isEmpty(filter)) {
-                boolean pass = cctx.isAllLocked(this, filter);
-
-                if (!pass) {
-                    if (expiryPlc != null && !readFromStore && !cctx.putIfAbsentFilter(filter) && hasValueUnlocked())
-                        updateTtlUnlocked(expiryPlc);
-
-                    Object val = retval ?
-                        cctx.cacheObjectContext().unwrapBinaryIfNeeded(CU.value(old, cctx, false), keepBinary, false, null)
-                        : null;
-
-                    return new T3<>(false, val, null);
-                }
-            }
-
-            String transformCloClsName = null;
-
-            CacheObject updated;
-
-            Object key0 = null;
-            Object updated0 = null;
-
-            // Calculate new value.
-            if (op == GridCacheOperation.TRANSFORM) {
-                transformCloClsName = EntryProcessorResourceInjectorProxy.unwrap(writeObj).getClass().getName();
-
-                EntryProcessor<Object, Object, ?> entryProcessor = (EntryProcessor<Object, Object, ?>)writeObj;
-
-                assert entryProcessor != null;
-
-                CacheInvokeEntry<Object, Object> entry = new CacheInvokeEntry<>(key, old, version(), keepBinary, this);
-
-                IgniteThread.onEntryProcessorEntered(false);
-
-                try {
-                    Object computed = entryProcessor.process(entry, invokeArgs);
-
-                    transformOp = true;
-
-                    if (entry.modified()) {
-                        updated0 = cctx.unwrapTemporary(entry.getValue());
-
-                        updated = cctx.toCacheObject(updated0);
-
-                        cctx.validateKeyAndValue(key, updated);
-                    }
-                    else
-                        updated = old;
-
-                    key0 = entry.key();
-
-                    invokeRes = computed != null ? CacheInvokeResult.fromResult(cctx.unwrapTemporary(computed)) : null;
-                }
-                catch (Exception e) {
-                    updated = old;
-
-                    invokeRes = CacheInvokeResult.fromError(e);
-                }
-                finally {
-                    IgniteThread.onEntryProcessorLeft();
-                }
-
-                if (!entry.modified()) {
-                    if (expiryPlc != null && !readFromStore && hasValueUnlocked())
-                        updateTtlUnlocked(expiryPlc);
-
-                    updateMetrics(READ, metrics, transformOp, old != null);
-
-                    return new GridTuple3<>(false, null, invokeRes);
-                }
-            }
-            else
-                updated = (CacheObject)writeObj;
-
-            op = updated == null ? DELETE : UPDATE;
-
-            if (intercept) {
-                CacheLazyEntry e;
-
-                if (op == UPDATE) {
-                    updated0 = value(updated0, updated, keepBinary, false);
-
-                    e = new CacheLazyEntry(cctx, key, key0, old, old0, keepBinary);
-
-                    Object interceptorVal = cctx.config().getInterceptor().onBeforePut(e, updated0);
-
-                    if (interceptorVal == null)
-                        return new GridTuple3<>(false, cctx.unwrapTemporary(value(old0, old, keepBinary, false)), invokeRes);
-                    else {
-                        updated0 = cctx.unwrapTemporary(interceptorVal);
-
-                        updated = cctx.toCacheObject(updated0);
-                    }
-                }
-                else {
-                    e = new CacheLazyEntry(cctx, key, key0, old, old0, keepBinary);
-
-                    interceptorRes = cctx.config().getInterceptor().onBeforeRemove(e);
-
-                    if (cctx.cancelRemove(interceptorRes))
-                        return new GridTuple3<>(false, cctx.unwrapTemporary(interceptorRes.get2()), invokeRes);
-                }
-
-                key0 = e.key();
-                old0 = e.value();
-            }
-
-            boolean hadVal = hasValueUnlocked();
-
-            long ttl = CU.TTL_ETERNAL;
-            long expireTime = CU.EXPIRE_TIME_ETERNAL;
-
-            if (op == UPDATE) {
-                if (expiryPlc != null) {
-                    ttl = CU.toTtl(hadVal ? expiryPlc.getExpiryForUpdate() : expiryPlc.getExpiryForCreation());
-
-                    if (ttl == CU.TTL_NOT_CHANGED) {
-                        ttl = ttlExtras();
-                        expireTime = expireTimeExtras();
-                    }
-                    else if (ttl != CU.TTL_ZERO)
-                        expireTime = CU.toExpireTime(ttl);
-                }
-                else {
-                    ttl = ttlExtras();
-                    expireTime = expireTimeExtras();
-                }
-            }
-
-            if (ttl == CU.TTL_ZERO) {
-                op = DELETE;
-
-                //If time expired no transformation needed.
-                transformOp = false;
-            }
-
-            // Try write-through.
-            if (op == UPDATE) {
-                // Detach value before index update.
-                updated = cctx.kernalContext().cacheObjects().prepareForCache(updated, cctx);
-
-                if (writeThrough)
-                    // Must persist inside synchronization in non-tx mode.
-                    cctx.store().put(null, key, updated, ver);
-
-                storeValue(updated, expireTime, ver);
-
-                assert ttl != CU.TTL_ZERO;
-
-                update(updated, expireTime, ttl, ver, true);
-
-                logUpdate(op, updated, ver, expireTime, 0, true);
-
-                if (evt) {
-                    CacheObject evtOld = null;
-
-                    if (transformCloClsName != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
-                        evtOld = cctx.unwrapTemporary(old);
-
-                        cctx.events().addEvent(partition(), key, cctx.localNodeId(), null, null,
-                            (GridCacheVersion)null, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld,
-                            evtOld != null || hadVal, transformCloClsName, taskName, keepBinary);
-                    }
-
-                    if (cctx.events().isRecordable(EVT_CACHE_OBJECT_PUT)) {
-                        if (evtOld == null)
-                            evtOld = cctx.unwrapTemporary(old);
-
-                        cctx.events().addEvent(partition(), key, cctx.localNodeId(), null, null,
-                            (GridCacheVersion)null, EVT_CACHE_OBJECT_PUT, updated, updated != null, evtOld,
-                            evtOld != null || hadVal, null, taskName, keepBinary);
-                    }
-                }
-            }
-            else {
-                if (writeThrough)
-                    // Must persist inside synchronization in non-tx mode.
-                    cctx.store().remove(null, key);
-
-                removeValue();
-
-                update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, ver, true);
-
-                logUpdate(op, null, ver, CU.EXPIRE_TIME_ETERNAL, 0, true);
-
-                if (evt) {
-                    CacheObject evtOld = null;
-
-                    if (transformCloClsName != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ))
-                        cctx.events().addEvent(partition(), key, cctx.localNodeId(), null, null,
-                            (GridCacheVersion)null, EVT_CACHE_OBJECT_READ, evtOld, evtOld != null || hadVal, evtOld,
-                            evtOld != null || hadVal, transformCloClsName, taskName, keepBinary);
-
-                    if (cctx.events().isRecordable(EVT_CACHE_OBJECT_REMOVED)) {
-                        if (evtOld == null)
-                            evtOld = cctx.unwrapTemporary(old);
-
-                        cctx.events().addEvent(partition(), key, cctx.localNodeId(), null, null, (GridCacheVersion)null,
-                            EVT_CACHE_OBJECT_REMOVED, null, false, evtOld, evtOld != null || hadVal, null,
-                            taskName, keepBinary);
-                    }
-                }
-
-                res = hadVal;
-            }
-
-            if (res)
-                updateMetrics(op, metrics, transformOp, old != null);
-            else if (op == DELETE && transformOp)
-                cctx.cache().metrics0().onInvokeRemove(old != null);
-
-            if (lsnrCol != null) {
-                long updateCntr = nextPartitionCounter(AffinityTopologyVersion.NONE, true, false, null);
-
-                cctx.continuousQueries().onEntryUpdated(
-                    lsnrCol,
-                    key,
-                    val,
-                    old,
-                    internal,
-                    partition(),
-                    true,
-                    false,
-                    updateCntr,
-                    null,
-                    AffinityTopologyVersion.NONE);
-
-                onUpdateFinished(updateCntr);
-            }
-
-            if (intercept) {
-                if (op == UPDATE)
-                    cctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(cctx, key, key0, updated, updated0, keepBinary, 0L));
-                else
-                    cctx.config().getInterceptor().onAfterRemove(new CacheLazyEntry(cctx, key, key0, old, old0, keepBinary, 0L));
-            }
-
-            updatePlatformCache(op == UPDATE ? updated : null, cctx.affinity().affinityTopologyVersion());
-        }
-        finally {
-            unlockEntry();
-            unlockListenerReadLock();
-        }
-
-        return new GridTuple3<>(res,
-            cctx.unwrapTemporary(interceptorRes != null ?
-                interceptorRes.get2() :
-                cctx.cacheObjectContext().unwrapBinaryIfNeeded(old, keepBinary, false, null)),
-            invokeRes);
-    }
-
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public GridCacheUpdateAtomicResult innerUpdate(
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index da1ccc2772e..6a35091d652 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -781,7 +781,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @return {@code True} if lock had been removed.
      */
     public boolean isRemoved(GridCacheContext cacheCtx, GridCacheVersion ver) {
-        return !cacheCtx.isNear() && !cacheCtx.isLocal() && ver != null && rmvLocks.contains(ver);
+        return !cacheCtx.isNear() && ver != null && rmvLocks.contains(ver);
     }
 
     /**
@@ -790,7 +790,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * @return {@code True} if added.
      */
     public boolean addRemoved(GridCacheContext cacheCtx, GridCacheVersion ver) {
-        if (cacheCtx.isNear() || cacheCtx.isLocal())
+        if (cacheCtx.isNear())
             return true;
 
         boolean ret = rmvLocks.add(ver);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index f45386604d0..63d5e52f0dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1378,12 +1378,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (oldest.id().equals(cctx.localNodeId())) {
             // Check rebalance state & send CacheAffinityChangeMessage if need.
             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                if (!grp.isLocal()) {
-                    GridDhtPartitionTopology top = grp.topology();
+                GridDhtPartitionTopology top = grp.topology();
 
-                    if (top != null)
-                        cctx.affinity().checkRebalanceState(top, grp.groupId());
-                }
+                if (top != null)
+                    cctx.affinity().checkRebalanceState(top, grp.groupId());
             }
 
             GridDhtPartitionsExchangeFuture lastFut = lastInitializedFut;
@@ -1541,39 +1539,37 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         Map<Integer, Map<Integer, Long>> partsSizes = new HashMap<>();
 
         for (CacheGroupContext grp : grps) {
-            if (!grp.isLocal()) {
-                if (exchId != null) {
-                    AffinityTopologyVersion startTopVer = grp.localStartVersion();
-
-                    if (startTopVer.compareTo(exchId.topologyVersion()) > 0)
-                        continue;
-                }
+            if (exchId != null) {
+                AffinityTopologyVersion startTopVer = grp.localStartVersion();
 
-                GridAffinityAssignmentCache affCache = grp.affinity();
+                if (startTopVer.compareTo(exchId.topologyVersion()) > 0)
+                    continue;
+            }
 
-                GridDhtPartitionFullMap locMap = grp.topology().partitionMap(true);
+            GridAffinityAssignmentCache affCache = grp.affinity();
 
-                if (locMap != null)
-                    addFullPartitionsMap(m, dupData, compress, grp.groupId(), locMap, affCache.similarAffinityKey());
+            GridDhtPartitionFullMap locMap = grp.topology().partitionMap(true);
 
-                Map<Integer, Long> partSizesMap = grp.topology().globalPartSizes();
+            if (locMap != null)
+                addFullPartitionsMap(m, dupData, compress, grp.groupId(), locMap, affCache.similarAffinityKey());
 
-                if (!partSizesMap.isEmpty())
-                    partsSizes.put(grp.groupId(), partSizesMap);
+            Map<Integer, Long> partSizesMap = grp.topology().globalPartSizes();
 
-                if (exchId != null) {
-                    CachePartitionFullCountersMap cntrsMap = grp.topology().fullUpdateCounters();
+            if (!partSizesMap.isEmpty())
+                partsSizes.put(grp.groupId(), partSizesMap);
 
-                    if (newCntrMap)
-                        m.addPartitionUpdateCounters(grp.groupId(), cntrsMap);
-                    else {
-                        m.addPartitionUpdateCounters(grp.groupId(),
-                            CachePartitionFullCountersMap.toCountersMap(cntrsMap));
-                    }
+            if (exchId != null) {
+                CachePartitionFullCountersMap cntrsMap = grp.topology().fullUpdateCounters();
 
-                    // Lost partitions can be skipped on node left or activation.
-                    m.addLostPartitions(grp.groupId(), grp.topology().lostPartitions());
+                if (newCntrMap)
+                    m.addPartitionUpdateCounters(grp.groupId(), cntrsMap);
+                else {
+                    m.addPartitionUpdateCounters(grp.groupId(),
+                        CachePartitionFullCountersMap.toCountersMap(cntrsMap));
                 }
+
+                // Lost partitions can be skipped on node left or activation.
+                m.addLostPartitions(grp.groupId(), grp.topology().lostPartitions());
             }
         }
 
@@ -1728,7 +1724,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         Map<Object, T2<Integer, GridPartitionStateMap>> dupData = new HashMap<>();
 
         for (CacheGroupContext grp : grps) {
-            if (!grp.isLocal() && (exchActions == null || !exchActions.cacheGroupStopping(grp.groupId()))) {
+            if (exchActions == null || !exchActions.cacheGroupStopping(grp.groupId())) {
                 GridDhtPartitionMap locMap = grp.topology().localPartitionMap();
 
                 addPartitionMap(m,
@@ -1957,12 +1953,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                     CacheGroupContext grp = cctx.cache().cacheGroup(grpId);
 
-                    GridDhtPartitionTopology top = null;
-
-                    if (grp == null)
-                        top = clientTops.get(grpId);
-                    else if (!grp.isLocal())
-                        top = grp.topology();
+                    GridDhtPartitionTopology top = grp == null ? clientTops.get(grpId) : grp.topology();
 
                     if (top != null) {
                         updated |= top.update(null,
@@ -1986,7 +1977,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 boolean hasMovingParts = false;
 
                 for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                    if (!grp.isLocal() && grp.topology().hasMovingPartitions()) {
+                    if (grp.topology().hasMovingPartitions()) {
                         hasMovingParts = true;
 
                         break;
@@ -2028,12 +2019,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     if (grp != null && !grp.topology().initialized())
                         continue;
 
-                    GridDhtPartitionTopology top = null;
-
-                    if (grp == null)
-                        top = clientTops.get(grpId);
-                    else if (!grp.isLocal())
-                        top = grp.topology();
+                    GridDhtPartitionTopology top = grp == null ? clientTops.get(grpId) : grp.topology();
 
                     if (top != null) {
                         updated |= top.update(null, entry.getValue(), false);
@@ -2203,12 +2189,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      */
     public void checkRebalanceState() {
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (!grp.isLocal()) {
-                GridDhtPartitionTopology top = grp.topology();
+            GridDhtPartitionTopology top = grp.topology();
 
-                if (top != null)
-                    cctx.affinity().checkRebalanceState(top, grp.groupId());
-            }
+            if (top != null)
+                cctx.affinity().checkRebalanceState(top, grp.groupId());
         }
     }
 
@@ -2601,9 +2585,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         int affDumpCnt = 0;
 
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal())
-                continue;
-
             GridCachePreloader preloader = grp.preloader();
 
             if (preloader != null)
@@ -3240,9 +3221,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     boolean preloadFinished = true;
 
                     for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                        if (grp.isLocal())
-                            continue;
-
                         preloadFinished &= grp.preloader() != null && grp.preloader().syncFuture().isDone();
 
                         if (!preloadFinished)
@@ -3456,12 +3434,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             // same event.
                             boolean changed = false;
 
-                            for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                                if (grp.isLocal())
-                                    continue;
-
+                            for (CacheGroupContext grp : cctx.cache().cacheGroups())
                                 changed |= grp.topology().afterExchange(exchFut);
-                            }
 
                             if (!cctx.kernalContext().clientNode() && changed) {
                                 if (log.isDebugEnabled())
@@ -3533,7 +3507,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             }
                             else {
                                 resVer = resVer == null ? assignsSet.stream()
-                                    .filter(g -> !g.isLocal())
                                     .map(g -> g.topology().readyTopologyVersion())
                                     .filter(Objects::nonNull)
                                     .findFirst()
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 98c00118710..d3354d7fa9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -58,8 +58,6 @@ import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheExistsException;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.AffinityFunctionContext;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cluster.ClusterNode;
@@ -110,8 +108,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearAtom
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTransactionalCache;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrManager;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
-import org.apache.ignite.internal.processors.cache.local.GridLocalCache;
-import org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache;
 import org.apache.ignite.internal.processors.cache.mvcc.DeadlockDetectionManager;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccCachingManager;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
@@ -134,7 +130,6 @@ import org.apache.ignite.internal.processors.cache.persistence.snapshot.Snapshot
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager;
-import org.apache.ignite.internal.processors.cache.query.GridCacheLocalQueryManager;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
@@ -200,7 +195,6 @@ import org.apache.ignite.spi.systemview.view.CachePagesListView;
 import org.apache.ignite.spi.systemview.view.PartitionStateView;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static java.lang.String.format;
 import static java.util.Arrays.asList;
 import static java.util.Objects.isNull;
@@ -214,7 +208,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
@@ -499,7 +492,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("IfMayBeConditional")
     private Collection<GridCacheManager> dhtExcludes(GridCacheContext ctx) {
-        if (ctx.config().getCacheMode() == LOCAL || !isNearEnabled(ctx))
+        if (!isNearEnabled(ctx))
             return Collections.emptyList();
         else
             return F.asList(ctx.queries(), ctx.continuousQueries(), ctx.store());
@@ -1299,9 +1292,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         CacheEvictionManager evictMgr = (nearEnabled || cfg.isOnheapCacheEnabled())
             ? new GridCacheEvictionManager()
             : new CacheOffheapEvictionManager();
-        GridCacheQueryManager qryMgr = cfg.getCacheMode() == LOCAL
-            ? new GridCacheLocalQueryManager()
-            : new GridCacheDistributedQueryManager();
+        GridCacheQueryManager qryMgr = new GridCacheDistributedQueryManager();
         CacheContinuousQueryManager contQryMgr = new CacheContinuousQueryManager();
         CacheDataStructuresManager dataStructuresMgr = new CacheDataStructuresManager();
         GridCacheTtlManager ttlMgr = new GridCacheTtlManager();
@@ -1355,27 +1346,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheAdapter cache = null;
 
         switch (cfg.getCacheMode()) {
-            case LOCAL: {
-                switch (cfg.getAtomicityMode()) {
-                    case TRANSACTIONAL:
-                    case TRANSACTIONAL_SNAPSHOT: {
-                        cache = new GridLocalCache(cacheCtx);
-
-                        break;
-                    }
-                    case ATOMIC: {
-                        cache = new GridLocalAtomicCache(cacheCtx);
-
-                        break;
-                    }
-
-                    default: {
-                        assert false : "Invalid cache atomicity mode: " + cfg.getAtomicityMode();
-                    }
-                }
-
-                break;
-            }
             case PARTITIONED:
             case REPLICATED: {
                 if (nearEnabled) {
@@ -1439,7 +1409,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
          * Create DHT cache.
          * ================
          */
-        if (cfg.getCacheMode() != LOCAL && nearEnabled) {
+        if (nearEnabled) {
             /*
              * Specifically don't create the following managers
              * here and reuse the one from Near cache:
@@ -1620,10 +1590,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * started.
      *
      * @param start Start cache.
-     * @param inclLoc Include local caches.
      * @return Cache or {@code null} if there is no suitable cache.
      */
-    public IgniteCacheProxy<?, ?> getOrStartPublicCache(boolean start, boolean inclLoc) throws IgniteCheckedException {
+    public IgniteCacheProxy<?, ?> getOrStartPublicCache(boolean start) throws IgniteCheckedException {
         // Try to find started cache first.
         for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
             if (!e.getValue().context().userCache())
@@ -1633,15 +1602,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             String cacheName = ccfg.getName();
 
-            if ((inclLoc || ccfg.getCacheMode() != LOCAL))
-                return publicJCache(cacheName);
+            return publicJCache(cacheName);
         }
 
         if (start) {
             for (Map.Entry<String, DynamicCacheDescriptor> e : cachesInfo.registeredCaches().entrySet()) {
                 DynamicCacheDescriptor desc = e.getValue();
 
-                if (!desc.cacheType().userCache() || desc.cacheConfiguration().getCacheMode() == LOCAL)
+                if (!desc.cacheType().userCache())
                     continue;
 
                 CacheConfiguration ccfg = desc.cacheConfiguration();
@@ -2014,8 +1982,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         AffinityTopologyVersion exchTopVer,
         boolean disabledAfterStart
     ) throws IgniteCheckedException {
-        desc = enricher().enrich(desc,
-            desc.cacheConfiguration().getCacheMode() == LOCAL || isLocalAffinity(desc.cacheConfiguration()));
+        desc = enricher().enrich(desc, isLocalAffinity(desc.cacheConfiguration()));
 
         CacheConfiguration startCfg = desc.cacheConfiguration();
 
@@ -2153,9 +2120,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheAdapter cacheAdapter : caches.values()) {
             GridCacheContext cacheContext = cacheAdapter.context();
 
-            if (cacheContext.isLocal())
-                continue;
-
             if (cacheContext.isRecoveryMode()) {
                 assert !isLocalAffinity(cacheContext.config())
                     : "Cache " + cacheAdapter.context() + " is still in recovery mode after start, but not activated.";
@@ -2178,12 +2142,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         @Nullable NearCacheConfiguration reqNearCfg,
         CacheConfiguration ccfg
     ) {
-        if (ccfg.getCacheMode() == LOCAL) {
-            ccfg.setNearConfiguration(null);
-
-            return true;
-        }
-
         if (isLocalAffinity(desc.cacheConfiguration()))
             return true;
 
@@ -2279,7 +2237,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         cacheCtx.initConflictResolver();
 
-        if (cfg.getCacheMode() != LOCAL && GridCacheUtils.isNearEnabled(cfg)) {
+        if (GridCacheUtils.isNearEnabled(cfg)) {
             GridCacheContext<?, ?> dhtCtx = cacheCtx.near().dht().context();
 
             // Start DHT managers.
@@ -2528,7 +2486,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         boolean needToStart = (dataRegion != null)
             && (cacheType != CacheType.USER
                 || (sharedCtx.isLazyMemoryAllocation(dataRegion)
-                    && (!cacheObjCtx.kernalContext().clientNode() || cfg.getCacheMode() == LOCAL)));
+                    && !cacheObjCtx.kernalContext().clientNode()));
 
         if (needToStart)
             dataRegion.pageMemory().start();
@@ -4017,9 +3975,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         sharedCtx.tm().checkEmptyTransactions(
             () -> format(CHECK_EMPTY_TRANSACTIONS_ERROR_MSG_FORMAT, cacheName, "dynamicCloseCache"));
 
-        if (proxy.context().isLocal())
-            return dynamicDestroyCache(cacheName, false, true, false, null);
-
         return startClientCacheChange(null, Collections.singleton(cacheName));
     }
 
@@ -5260,8 +5215,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (IgniteInternalCache cache : caches) {
             cache.context().statisticsEnabled(enabled);
 
-            if (!cache.context().isLocal())
-                globalCaches.add(cache.name());
+            globalCaches.add(cache.name());
         }
 
         if (globalCaches.isEmpty())
@@ -5288,10 +5242,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         Collection<String> globalCaches = new HashSet<>(U.capacity(caches.size()));
 
-        for (IgniteInternalCache cache : caches) {
-            if (!cache.context().isLocal())
-                globalCaches.add(cache.name());
-        }
+        for (IgniteInternalCache cache : caches)
+            globalCaches.add(cache.name());
 
         if (globalCaches.isEmpty())
             return;
@@ -5920,54 +5872,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
     }
 
-    /**
-     * The reason why this class is not removed is backward compatibility
-     * in the case of using local caches with native persistence.
-     */
-    @Deprecated
-    private static class LocalAffinityFunction implements AffinityFunction {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private static final org.apache.ignite.internal.processors.affinity.LocalAffinityFunction DELEGATE =
-            new org.apache.ignite.internal.processors.affinity.LocalAffinityFunction();
-
-        /**
-         * Should not be directly used.
-         */
-        LocalAffinityFunction() throws IgniteCheckedException {
-            throw new IgniteCheckedException("This class should not be directly instantiated. Please use "
-                + org.apache.ignite.internal.processors.affinity.LocalAffinityFunction.class.getCanonicalName()
-                + " instead.");
-        }
-
-        /** {@inheritDoc} */
-        @Override public List<List<ClusterNode>> assignPartitions(AffinityFunctionContext affCtx) {
-            return DELEGATE.assignPartitions(affCtx);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void reset() {
-            DELEGATE.reset();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int partitions() {
-            return DELEGATE.partitions();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int partition(Object key) {
-            return DELEGATE.partition(key);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void removeNode(UUID nodeId) {
-            DELEGATE.removeNode(nodeId);
-        }
-    }
-
     /**
      * Creation of a string representation of the top (descending) partitions, the processing of which took the most time.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index d2ea1068096..c9be5195e71 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -110,7 +110,7 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
 
         cctx.shared().ttl().register(this);
 
-        pendingEntries = (!cctx.isLocal() && cctx.config().getNearConfiguration() != null) ? new GridConcurrentSkipListSetEx() : null;
+        pendingEntries = (cctx.config().getNearConfiguration() != null) ? new GridConcurrentSkipListSetEx() : null;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 96123a3856d..a248033e9dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -68,7 +68,6 @@ import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.affinity.LocalAffinityFunction;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockCancelledException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
@@ -116,10 +115,8 @@ import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionRollbackException;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static java.util.Objects.nonNull;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.ASYNC;
@@ -496,9 +493,6 @@ public class GridCacheUtils {
      */
     @SuppressWarnings("SimplifiableIfStatement")
     public static boolean isNearEnabled(CacheConfiguration cfg) {
-        if (cfg.getCacheMode() == LOCAL)
-            return false;
-
         return cfg.getNearConfiguration() != null;
     }
 
@@ -1727,23 +1721,13 @@ public class GridCacheUtils {
 
                 cfg.setAffinity(aff);
             }
-            else if (cfg.getCacheMode() == REPLICATED) {
+            else {
                 RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false, 512);
 
                 cfg.setAffinity(aff);
 
                 cfg.setBackups(Integer.MAX_VALUE);
             }
-            else
-                cfg.setAffinity(new LocalAffinityFunction());
-        }
-        else {
-            if (cfg.getCacheMode() == LOCAL && !(cfg.getAffinity() instanceof LocalAffinityFunction)) {
-                cfg.setAffinity(new LocalAffinityFunction());
-
-                U.warn(log, "AffinityFunction configuration parameter will be ignored for local cache" +
-                    " [cacheName=" + U.maskName(cfg.getName()) + ']');
-            }
         }
 
         validateKeyConfigiration(cfg.getGroupName(), cfg.getName(), cfg.getKeyConfiguration(), log, true);
@@ -2161,7 +2145,7 @@ public class GridCacheUtils {
             if (cctx == null)
                 throw new CacheException("Failed to find cache.");
 
-            if (!cctx.isLocal() && !cctx.isReplicated())
+            if (!cctx.isReplicated())
                 return cctx;
         }
 
@@ -2180,7 +2164,7 @@ public class GridCacheUtils {
             if (cctx == null)
                 throw new CacheException("Failed to find cache.");
 
-            if (!cctx.isLocal() && !cctx.isReplicated())
+            if (!cctx.isReplicated())
                 return cctx;
         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 33f18528b21..e6f78780e5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -121,7 +121,6 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static java.lang.Boolean.TRUE;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
@@ -195,9 +194,6 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
 
             try {
                 initDataStructures();
-
-                if (grp.isLocal())
-                    locCacheDataStore = createCacheDataStore(0);
             }
             finally {
                 ctx.database().checkpointReadUnlock();
@@ -320,17 +316,11 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
      * @return Data store.
      */
     @Nullable private CacheDataStore dataStore(GridCacheContext<?, ?> cctx, KeyCacheObject key) {
-        if (grp.isLocal())
-            return locCacheDataStore;
-
         return dataStore(cctx.affinity().partition(key), false);
     }
 
     /** {@inheritDoc} */
     @Override public CacheDataStore dataStore(@Nullable GridDhtLocalPartition part) {
-        if (grp.isLocal())
-            return locCacheDataStore;
-
         assert part != null;
 
         return part.dataStore();
@@ -342,9 +332,6 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
      * @return Related partition cache data store or {@code null} if partition haven't been initialized.
      */
     @Nullable private CacheDataStore dataStore(int partId, boolean includeRenting) {
-        if (grp.isLocal())
-            return locCacheDataStore;
-
         GridDhtLocalPartition part = grp.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, includeRenting);
 
         return part == null ? null : part.dataStore();
@@ -372,21 +359,14 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         boolean backup,
         AffinityTopologyVersion topVer
     ) throws IgniteCheckedException {
-        if (grp.isLocal())
-            if (primary)
-                return cacheEntriesCount(cacheId, 0);
-            else
-                return 0L;
-        else {
-            long cnt = 0;
+        long cnt = 0;
 
-            Iterator<CacheDataStore> it = cacheData(primary, backup, topVer);
+        Iterator<CacheDataStore> it = cacheData(primary, backup, topVer);
 
-            while (it.hasNext())
-                cnt += it.next().cacheSize(cacheId);
+        while (it.hasNext())
+            cnt += it.next().cacheSize(cacheId);
 
-            return cnt;
-        }
+        return cnt;
     }
 
     /** {@inheritDoc} */
@@ -408,8 +388,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     private Iterable<CacheDataStore> cacheDataStores(
         IgnitePredicate<GridDhtLocalPartition> filter
     ) {
-        return grp.isLocal() ? Collections.singletonList(locCacheDataStore) :
-            F.iterator(grp.topology().currentLocalPartitions(), GridDhtLocalPartition::dataStore, true,
+        return F.iterator(grp.topology().currentLocalPartitions(), GridDhtLocalPartition::dataStore, true,
                 filter, p -> !p.dataStore().destroyed());
     }
 
@@ -422,9 +401,6 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     private Iterator<CacheDataStore> cacheData(boolean primary, boolean backup, AffinityTopologyVersion topVer) {
         assert primary || backup;
 
-        if (grp.isLocal())
-            return singletonIterator(locCacheDataStore);
-
         IgnitePredicate<GridDhtLocalPartition> filter;
 
         if (primary && backup)
@@ -628,7 +604,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         throws IgniteCheckedException {
         KeyCacheObject key = entry.key();
 
-        assert grp.isLocal() || entry.localPartition() != null : entry;
+        assert entry.localPartition() != null : entry;
 
         return dataStore(entry.localPartition()).find(entry.context(), key);
     }
@@ -696,9 +672,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     @Override public void clearCache(GridCacheContext cctx, boolean readers) {
         GridCacheVersion obsoleteVer = null;
 
-        try (GridCloseableIterator<CacheDataRow> it = grp.isLocal() ?
-            iterator(cctx.cacheId(), cacheDataStores().iterator(), null, null) :
-            evictionSafeIterator(cctx.cacheId(), cacheDataStores().iterator())) {
+        try (GridCloseableIterator<CacheDataRow> it = evictionSafeIterator(cctx.cacheId(), cacheDataStores().iterator())) {
             while (it.hasNext()) {
                 cctx.shared().database().checkpointReadLock();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
index 684300a9dfb..007552b7f52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
@@ -380,16 +380,10 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
 
         try {
             if (isAsync()) {
-                if (ctx.cache().isLocal())
-                    setFuture(ctx.cache().localLoadCacheAsync(p, args));
-                else
-                    setFuture(ctx.cache().globalLoadCacheAsync(p, args));
+                setFuture(ctx.cache().globalLoadCacheAsync(p, args));
             }
             else {
-                if (ctx.cache().isLocal())
-                    ctx.cache().localLoadCache(p, args);
-                else
-                    ctx.cache().globalLoadCache(p, args);
+                ctx.cache().globalLoadCache(p, args);
             }
         }
         catch (IgniteCheckedException | IgniteException e) {
@@ -403,9 +397,6 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
         GridCacheContext<K, V> ctx = getContextSafe();
 
         try {
-            if (ctx.cache().isLocal())
-                return (IgniteFuture<Void>)createFuture(ctx.cache().localLoadCacheAsync(p, args));
-
             return (IgniteFuture<Void>)createFuture(ctx.cache().globalLoadCacheAsync(p, args));
         }
         catch (IgniteCheckedException | IgniteException e) {
@@ -647,10 +638,7 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
     private ClusterGroup projection(boolean loc) {
         GridCacheContext<K, V> ctx = getContextSafe();
 
-        if (loc || ctx.isLocal())
-            return ctx.kernalContext().grid().cluster().forLocal();
-
-        return null;
+        return loc ? ctx.kernalContext().grid().cluster().forLocal() : null;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index de7d9a3382e..4ab8963cfb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -1447,8 +1447,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public int nearSize();
 
     /**
-     * Gets the number of all primary entries cached on this node. For {@link CacheMode#LOCAL} non-distributed
-     * cache mode, this method is identical to {@link #size()}.
+     * Gets the number of all primary entries cached on this node.
      * <p>
      * For {@link CacheMode#PARTITIONED} and {@link CacheMode#REPLICATED} modes, this method will
      * return number of primary entries cached on this node (excluding any backups). The complexity of
@@ -1461,8 +1460,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public int primarySize();
 
     /**
-     * Gets the number of all primary entries cached on this node as a long value. For {@link CacheMode#LOCAL}
-     * non-distributed cache mode, this method is identical to {@link #size()}.
+     * Gets the number of all primary entries cached on this node as a long value.
      * <p>
      * For {@link CacheMode#PARTITIONED} and {@link CacheMode#REPLICATED} modes, this method will
      * return number of primary entries cached on this node (excluding any backups). The complexity of
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java
index 4e8f05f600b..1506dcfdf55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java
@@ -51,7 +51,6 @@ import org.apache.ignite.internal.IgniteFeatures;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.cluster.DetachedClusterNode;
-import org.apache.ignite.internal.processors.affinity.LocalAffinityFunction;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
 import org.apache.ignite.internal.processors.query.QuerySchemaPatch;
@@ -71,11 +70,9 @@ import org.apache.ignite.spi.encryption.EncryptionSpi;
 import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.apache.ignite.spi.indexing.noop.NoopIndexingSpi;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
 import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
@@ -262,10 +259,6 @@ public class ValidationOnNodeJoinUtils {
             throw new IgniteCheckedException("DataRegion for client caches must be explicitly configured " +
                 "on client node startup. Use DataStorageConfiguration to configure DataRegion.");
 
-        if (cc.getCacheMode() == LOCAL && !cc.getAffinity().getClass().equals(LocalAffinityFunction.class))
-            U.warn(log, "AffinityFunction configuration parameter will be ignored for local cache [cacheName=" +
-                U.maskName(cc.getName()) + ']');
-
         if (cc.getAffinity().partitions() > CacheConfiguration.MAX_PARTITIONS_COUNT)
             throw new IgniteCheckedException("Cannot have more than " + CacheConfiguration.MAX_PARTITIONS_COUNT +
                 " partitions [cacheName=" + cc.getName() + ", partitions=" + cc.getAffinity().partitions() + ']');
@@ -298,9 +291,6 @@ public class ValidationOnNodeJoinUtils {
                 ", affFunction=" + cc.getAffinity() + ", cacheName=" + cc.getName() + ']');
 
         if (cc.getAtomicityMode() == TRANSACTIONAL_SNAPSHOT) {
-            apply(assertParam, cc.getCacheMode() != LOCAL,
-                "LOCAL cache mode cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
-
             apply(assertParam, cc.getNearConfiguration() == null,
                 "near cache cannot be used with TRANSACTIONAL_SNAPSHOT atomicity mode");
 
@@ -571,13 +561,8 @@ public class ValidationOnNodeJoinUtils {
      * @return {@code true} if cache is starting on client node and this node is affinity node for the cache.
      */
     private static boolean storesLocallyOnClient(IgniteConfiguration c, CacheConfiguration cc, GridKernalContext ctx) {
-        if (c.isClientMode() && c.getDataStorageConfiguration() == null) {
-            if (cc.getCacheMode() == LOCAL)
-                return true;
-
+        if (c.isClientMode() && c.getDataStorageConfiguration() == null)
             return ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cc.getName());
-
-        }
         else
             return false;
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
index 4ecd871dc81..95886263f10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
@@ -30,7 +30,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
@@ -62,7 +61,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.thread.OomExceptionHandler;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISABLE_WAL_DURING_REBALANCING;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PENDING_TX_TRACKER_ENABLED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_WAL;
@@ -379,9 +377,6 @@ public class WalStateManager extends GridCacheSharedManagerAdapter {
                     "provided [group=" + grpDesc.groupName() + ", missingCaches=" + grpCaches + ']');
             }
 
-            if (grpDesc.config().getCacheMode() == CacheMode.LOCAL)
-                return errorFuture("WAL mode cannot be changed for LOCAL cache(s): " + cacheNames);
-
             // WAL mode change makes sense only for persistent groups.
             if (!grpDesc.persistenceEnabled())
                 return errorFuture("Cannot change WAL mode because persistence is not enabled for cache(s) [" +
@@ -437,7 +432,7 @@ public class WalStateManager extends GridCacheSharedManagerAdapter {
         Collection<CacheGroupContext> grpContexts = cctx.cache().cacheGroups();
 
         for (CacheGroupContext grp : grpContexts) {
-            if (grp.isLocal() || !grp.affinityNode() || !(grp.persistenceEnabled() || grp.cdcEnabled())
+            if (!grp.affinityNode() || !(grp.persistenceEnabled() || grp.cdcEnabled())
                 || !grp.localWalEnabled() || !grp.rebalanceEnabled() || !grp.shared().isRebalanceEnabled())
                 continue;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
index 1cdeb3f41f6..17554ee2248 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
@@ -188,12 +188,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
 
         AffinityTopologyVersion topVer = topologyVersion();
 
-        int nodesCnt;
-
-        if (!cctx.isLocal())
-            nodesCnt = cctx.discovery().cacheGroupAffinityNodes(cctx.groupId(), topVer).size();
-        else
-            nodesCnt = 1;
+        int nodesCnt = cctx.discovery().cacheGroupAffinityNodes(cctx.groupId(), topVer).size();
 
         // Must return empty map if no alive nodes present or keys is empty.
         Map<ClusterNode, Collection<K>> res = new HashMap<>(nodesCnt, 1.0f);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index abf7881a7c7..daca76804ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -321,7 +321,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
                         }
                     },
                     new QueueHeaderPredicate(),
-                    cctx.isLocal() || (cctx.isReplicated() && cctx.affinityNode()),
+                    cctx.isReplicated() && cctx.affinityNode(),
                     true,
                     false,
                     false);
@@ -430,15 +430,11 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
      * @throws IgniteCheckedException If failed.
      */
     private void removeSetData(IgniteUuid setId, AffinityTopologyVersion topVer) throws IgniteCheckedException {
-        boolean loc = cctx.isLocal();
-
         GridCacheAffinityManager aff = cctx.affinity();
 
-        if (!loc) {
-            aff.affinityReadyFuture(topVer).get();
+        aff.affinityReadyFuture(topVer).get();
 
-            cctx.preloader().syncFuture().get();
-        }
+        cctx.preloader().syncFuture().get();
 
         IgniteInternalCache<?, ?> cache = cctx.cache();
 
@@ -473,74 +469,67 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
     public void removeSetData(IgniteUuid id, boolean separated) throws IgniteCheckedException {
         assert id != null;
 
-        if (!cctx.isLocal()) {
-            while (true) {
-                AffinityTopologyVersion topVer = cctx.topologyVersionFuture().get();
+        while (true) {
+            AffinityTopologyVersion topVer = cctx.topologyVersionFuture().get();
 
-                Collection<ClusterNode> nodes = F.view(cctx.discovery().nodes(topVer), node -> !node.isDaemon());
+            Collection<ClusterNode> nodes = F.view(cctx.discovery().nodes(topVer), node -> !node.isDaemon());
 
-                try {
-                    cctx.closures().callAsyncNoFailover(BROADCAST,
-                        new BlockSetCallable(cctx.name(), id),
-                        nodes,
-                        true,
-                        0, false).get();
+            try {
+                cctx.closures().callAsyncNoFailover(BROADCAST,
+                    new BlockSetCallable(cctx.name(), id),
+                    nodes,
+                    true,
+                    0, false).get();
 
-                    // Separated cache will be destroyed after the set is blocked.
-                    if (separated)
-                        break;
+                // Separated cache will be destroyed after the set is blocked.
+                if (separated)
+                    break;
+            }
+            catch (IgniteCheckedException e) {
+                if (e.hasCause(ClusterTopologyCheckedException.class)) {
+                    if (log.isDebugEnabled())
+                        log.debug("RemoveSetData job failed, will retry: " + e);
+
+                    continue;
                 }
-                catch (IgniteCheckedException e) {
-                    if (e.hasCause(ClusterTopologyCheckedException.class)) {
-                        if (log.isDebugEnabled())
-                            log.debug("RemoveSetData job failed, will retry: " + e);
-
-                        continue;
-                    }
-                    else if (!pingNodes(nodes)) {
-                        if (log.isDebugEnabled())
-                            log.debug("RemoveSetData job failed and set data node left, will retry: " + e);
-
-                        continue;
-                    }
-                    else
-                        throw e;
+                else if (!pingNodes(nodes)) {
+                    if (log.isDebugEnabled())
+                        log.debug("RemoveSetData job failed and set data node left, will retry: " + e);
+
+                    continue;
                 }
+                else
+                    throw e;
+            }
 
-                Collection<ClusterNode> affNodes = CU.affinityNodes(cctx, topVer);
+            Collection<ClusterNode> affNodes = CU.affinityNodes(cctx, topVer);
 
-                try {
-                    cctx.closures().callAsyncNoFailover(BROADCAST,
-                        new RemoveSetDataCallable(cctx.name(), id, topVer),
-                        affNodes,
-                        true,
-                        0, false).get();
-                }
-                catch (IgniteCheckedException e) {
-                    if (e.hasCause(ClusterTopologyCheckedException.class)) {
-                        if (log.isDebugEnabled())
-                            log.debug("RemoveSetData job failed, will retry: " + e);
-
-                        continue;
-                    }
-                    else if (!pingNodes(affNodes)) {
-                        if (log.isDebugEnabled())
-                            log.debug("RemoveSetData job failed and set data node left, will retry: " + e);
-
-                        continue;
-                    }
-                    else
-                        throw e;
+            try {
+                cctx.closures().callAsyncNoFailover(BROADCAST,
+                    new RemoveSetDataCallable(cctx.name(), id, topVer),
+                    affNodes,
+                    true,
+                    0, false).get();
+            }
+            catch (IgniteCheckedException e) {
+                if (e.hasCause(ClusterTopologyCheckedException.class)) {
+                    if (log.isDebugEnabled())
+                        log.debug("RemoveSetData job failed, will retry: " + e);
+
+                    continue;
                 }
+                else if (!pingNodes(affNodes)) {
+                    if (log.isDebugEnabled())
+                        log.debug("RemoveSetData job failed and set data node left, will retry: " + e);
 
-                if (topVer.equals(cctx.topologyVersionFuture().get()))
-                    break;
+                    continue;
+                }
+                else
+                    throw e;
             }
-        }
-        else {
-            blockSet(id);
 
-            cctx.dataStructures().removeSetData(id, AffinityTopologyVersion.ZERO);
+            if (topVer.equals(cctx.topologyVersionFuture().get()))
+                break;
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
index 1f248a486e7..262c107f19b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFutureAdapter.java
@@ -56,8 +56,7 @@ public abstract class GridDhtTopologyFutureAdapter extends GridFutureAdapter<Aff
      * @return Validation result.
      */
     protected final CacheGroupValidation validateCacheGroup(CacheGroupContext grp, Collection<ClusterNode> topNodes) {
-        Collection<Integer> lostParts = grp.isLocal() ?
-            Collections.<Integer>emptyList() : grp.topology().lostPartitions();
+        Collection<Integer> lostParts = grp.topology().lostPartitions();
 
         boolean valid = true;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 6667af42699..e5008e6ac89 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -529,8 +529,7 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
                 }
 
                 // Send old value in case if rebalancing is not finished.
-                final boolean sndOldVal = !cacheCtx.isLocal() &&
-                    !cacheCtx.topology().rebalanceFinished(tx.topologyVersion());
+                final boolean sndOldVal = !cacheCtx.topology().rebalanceFinished(tx.topologyVersion());
 
                 if (sndOldVal) {
                     if (oldVal == null && !readOld) {
@@ -666,9 +665,6 @@ public final class GridDhtTxPrepareFuture extends GridCacheCompoundFuture<Ignite
         for (IgniteTxEntry txEntry : checkEntries) {
             GridCacheContext cacheCtx = txEntry.context();
 
-            if (cacheCtx.isLocal())
-                continue;
-
             GridDistributedCacheEntry entry = (GridDistributedCacheEntry)txEntry.cached();
 
             if (entry == null) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java
index 4d563447231..f475efa48f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxQueryEnlistFuture.java
@@ -145,7 +145,7 @@ public final class GridDhtTxQueryEnlistFuture extends GridDhtTxQueryAbstractEnli
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     private void checkPartitions(int[] parts) throws ClusterTopologyCheckedException {
-        if (cctx.isLocal() || !cctx.rebalanceEnabled())
+        if (!cctx.rebalanceEnabled())
             return;
 
         GridDhtPartitionTopology top = cctx.topology();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 05ea509edda..89096172a1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -47,7 +47,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterState;
@@ -130,7 +129,6 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.jetbrains.annotations.Nullable;
-
 import static java.util.Collections.emptySet;
 import static java.util.stream.Stream.concat;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_LONG_OPERATIONS_DUMP_TIMEOUT_LIMIT;
@@ -1222,9 +1220,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         try {
             if (crd != null) {
                 for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                    if (grp.isLocal())
-                        continue;
-
                     grp.topology().beforeExchange(this, !centralizedAff && !forceAffReassignment, false);
 
                     cctx.exchange().exchangerUpdateHeartbeat();
@@ -1244,9 +1239,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      */
     private void updateTopologies(boolean crd) throws IgniteCheckedException {
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal())
-                continue;
-
             GridClientPartitionTopology clientTop = cctx.exchange().clearClientTopology(grp.groupId());
 
             long updSeq = clientTop == null ? -1 : clientTop.lastUpdateSequence();
@@ -1636,9 +1628,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         assert !cctx.kernalContext().clientNode();
 
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal())
-                continue;
-
             cctx.exchange().exchangerBlockingSectionBegin();
 
             try {
@@ -1685,7 +1674,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         boolean topChanged = firstDiscoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT || affChangeMsg != null;
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (cacheCtx.isLocal() || cacheStopping(cacheCtx.cacheId()))
+            if (cacheStopping(cacheCtx.cacheId()))
                 continue;
 
             if (topChanged) {
@@ -1718,7 +1707,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         // Pre-create missing partitions using current affinity.
         if (!exchCtx.mergeExchanges() && !exchCtx.exchangeFreeSwitch()) {
             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                if (grp.isLocal() || cacheGroupStopping(grp.groupId()))
+                if (cacheGroupStopping(grp.groupId()))
                     continue;
 
                 // It is possible affinity is not initialized yet if node joins to cluster.
@@ -1780,9 +1769,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     cctx.kernalContext().pools().getSystemExecutorService(),
                     cctx.affinity().cacheGroups().values(),
                     desc -> {
-                        if (desc.config().getCacheMode() == CacheMode.LOCAL)
-                            return null;
-
                         CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
 
                         GridDhtPartitionTopology top = grp != null ? grp.topology() :
@@ -2094,9 +2080,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      */
     private void onLeft() {
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal())
-                continue;
-
             grp.preloader().pause();
 
             try {
@@ -2138,7 +2121,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
     /**
      * @param grpId Cache group ID to check.
-     * @return {@code True} if cache group us stopping by this exchange.
+     * @return {@code True} if cache group is stopping by this exchange.
      */
     private boolean cacheGroupStopping(int grpId) {
         return exchActions != null && exchActions.cacheGroupStopping(grpId);
@@ -2460,7 +2443,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 !cctx.kernalContext().clientNode() &&
                 (serverNodeDiscoveryEvent() || affChangeMsg != null)) {
                 for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                    if (!cacheCtx.affinityNode() || cacheCtx.isLocal())
+                    if (!cacheCtx.affinityNode())
                         continue;
 
                     cacheCtx.continuousQueries().flushOnExchangeDone(res);
@@ -2474,9 +2457,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     Collection<CacheGroupContext> grpToRefresh = U.newHashSet(cctx.cache().cacheGroups().size());
 
                     for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                        if (grp.isLocal())
-                            continue;
-
                         try {
                             if (grp.topology().initPartitionsWhenAffinityReady(res, this))
                                 grpToRefresh.add(grp);
@@ -2554,10 +2534,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                 cleanIdxRebuildFutures = false;
 
-                for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                    if (!grp.isLocal())
-                        grp.topology().onExchangeDone(this, grp.affinity().readyAffinity(res), false);
-                }
+                for (CacheGroupContext grp : cctx.cache().cacheGroups())
+                    grp.topology().onExchangeDone(this, grp.affinity().readyAffinity(res), false);
 
                 if (changedAffinity())
                     cctx.walState().disableGroupDurabilityForPreloading(this);
@@ -3664,9 +3642,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 cctx.kernalContext().pools().getSystemExecutorService(),
                 cctx.affinity().cacheGroups().values(),
                 desc -> {
-                    if (desc.config().getCacheMode() == CacheMode.LOCAL)
-                        return null;
-
                     CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
 
                     GridDhtPartitionTopology top = grp != null ? grp.topology() :
@@ -3696,9 +3671,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 cctx.kernalContext().pools().getSystemExecutorService(),
                 cctx.affinity().caches().values(),
                 desc -> {
-                    if (desc.cacheConfiguration().getCacheMode() == CacheMode.LOCAL)
-                        return null;
-
                     if (cacheNames.contains(desc.cacheName())) {
                         CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
 
@@ -3798,9 +3770,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             if (!exchCtx.mergeExchanges() && !crd.equals(events().discoveryCache().serverNodes().get(0))) {
                 for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                    if (grp.isLocal())
-                        continue;
-
                     // It is possible affinity is not initialized.
                     // For example, dynamic cache start failed.
                     if (grp.affinity().lastVersion().topologyVersion() > 0)
@@ -3892,9 +3861,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     cctx.kernalContext().pools().getSystemExecutorService(),
                     cctx.affinity().cacheGroups().values(),
                     desc -> {
-                        if (desc.config().getCacheMode() == CacheMode.LOCAL)
-                            return null;
-
                         CacheGroupContext grp = cctx.cache().cacheGroup(desc.groupId());
 
                         GridDhtPartitionTopology top = grp != null ? grp.topology() :
@@ -3971,10 +3937,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 timeBag.finishGlobalStage("Ideal affinity diff calculation (enforced)");
             }
 
-            for (CacheGroupContext grpCtx : cctx.cache().cacheGroups()) {
-                if (!grpCtx.isLocal())
-                    grpCtx.topology().applyUpdateCounters();
-            }
+            for (CacheGroupContext grpCtx : cctx.cache().cacheGroups())
+                grpCtx.topology().applyUpdateCounters();
 
             timeBag.finishGlobalStage("Apply update counters");
 
@@ -4180,20 +4144,16 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      *
      * @return Collection of non local cache group descriptors.
      */
-    private List<CacheGroupDescriptor> nonLocalCacheGroupDescriptors() {
-        return cctx.affinity().cacheGroups().values().stream()
-            .filter(grpDesc -> grpDesc.config().getCacheMode() != CacheMode.LOCAL)
-            .collect(Collectors.toList());
+    private List<CacheGroupDescriptor> cacheGroupDescriptors() {
+        return new ArrayList<>(cctx.affinity().cacheGroups().values());
     }
 
     /**
-     * Collects non local cache groups.
-     *
-     * @return Collection of non local cache groups.
+     * @return Collection of cache groups that are not being stopped by this exchange.
      */
-    private List<CacheGroupContext> nonLocalCacheGroups() {
+    private List<CacheGroupContext> nonStoppingCacheGroups() {
         return cctx.cache().cacheGroups().stream()
-            .filter(grp -> !grp.isLocal() && !cacheGroupStopping(grp.groupId()))
+            .filter(grp -> !cacheGroupStopping(grp.groupId()))
             .collect(Collectors.toList());
     }
 
@@ -4204,7 +4164,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         try {
             U.doInParallel(
                 cctx.kernalContext().pools().getSystemExecutorService(),
-                nonLocalCacheGroupDescriptors(),
+                cacheGroupDescriptors(),
                 grpDesc -> {
                     CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpDesc.groupId());
 
@@ -4260,7 +4220,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         try {
             U.doInParallel(
                 cctx.kernalContext().pools().getSystemExecutorService(),
-                nonLocalCacheGroupDescriptors(),
+                cacheGroupDescriptors(),
                 grpDesc -> {
                     CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpDesc.groupId());
 
@@ -4362,7 +4322,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             U.<CacheGroupContext, Void>doInParallelUninterruptibly(
                 parallelismLvl,
                 cctx.kernalContext().pools().getSystemExecutorService(),
-                nonLocalCacheGroups(),
+                nonStoppingCacheGroups(),
                 grp -> {
                     Set<Integer> parts;
 
@@ -4765,7 +4725,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         cctx.affinity().onServerJoinWithExchangeMergeProtocol(this, false);
 
                     for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                        if (grp.isLocal() || cacheGroupStopping(grp.groupId()))
+                        if (cacheGroupStopping(grp.groupId()))
                             continue;
 
                         grp.topology().beforeExchange(this, true, false);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionDefferedDeleteQueueCleanupTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionDefferedDeleteQueueCleanupTask.java
index f69fd46e56c..83c1c6ffb08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionDefferedDeleteQueueCleanupTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionDefferedDeleteQueueCleanupTask.java
@@ -72,7 +72,7 @@ public class PartitionDefferedDeleteQueueCleanupTask implements GridTimeoutObjec
             @Override public void run() {
                 try {
                     for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-                        if (!grp.isLocal() && grp.affinityNode()) {
+                        if (grp.affinityNode()) {
                             GridDhtPartitionTopology top = null;
 
                             try {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index bbda303910e..69fb67a8db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -40,7 +40,6 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTx
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.tracing.MTC;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
@@ -59,7 +58,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteReducer;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
 import static org.apache.ignite.transactions.TransactionState.PREPARED;
@@ -97,42 +95,10 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
         if (log.isDebugEnabled())
             log.debug("Transaction future received owner changed callback: " + entry);
 
-        if ((entry.context().isNear() || entry.context().isLocal()) && owner != null) {
+        if (entry.context().isNear() && owner != null) {
             IgniteTxEntry txEntry = tx.entry(entry.txKey());
 
             if (txEntry != null) {
-                if (entry.context().isLocal()) {
-                    GridCacheVersion serReadVer = txEntry.entryReadVersion();
-
-                    if (serReadVer != null) {
-                        GridCacheContext ctx = entry.context();
-
-                        while (true) {
-                            try {
-                                if (!entry.checkSerializableReadVersion(serReadVer)) {
-                                    Object key = entry.key().value(ctx.cacheObjectContext(), false);
-
-                                    IgniteTxOptimisticCheckedException err0 =
-                                        new IgniteTxOptimisticCheckedException(S.toString(
-                                            "Failed to prepare transaction, read/write conflict",
-                                            "key", key, true,
-                                            "cache", ctx.name(), false));
-
-                                    ERR_UPD.compareAndSet(this, null, err0);
-                                }
-
-                                break;
-                            }
-                            catch (GridCacheEntryRemovedException ignored) {
-                                entry = ctx.cache().entryEx(entry.key(), tx.topologyVersion());
-
-                                txEntry.cached(entry);
-                            }
-                        }
-
-                    }
-                }
-
                 if (keyLockFut != null)
                     keyLockFut.onKeyLocked(entry.txKey());
 
@@ -625,9 +591,7 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
     ) {
         GridCacheContext cacheCtx = entry.context();
 
-        List<ClusterNode> nodes = cacheCtx.isLocal() ?
-            cacheCtx.affinity().nodesByKey(entry.key(), topVer) :
-            cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer);
+        List<ClusterNode> nodes = cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer);
 
         if (F.isEmpty(nodes)) {
             onDone(new ClusterTopologyServerNotFoundException("Failed to map keys to nodes " +
@@ -652,12 +616,10 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
         // Must re-initialize cached entry while holding topology lock.
         if (cacheCtx.isNear())
             entry.cached(cacheCtx.nearTx().entryExx(entry.key(), topVer));
-        else if (!cacheCtx.isLocal())
-            entry.cached(cacheCtx.colocated().entryExx(entry.key(), topVer, true));
         else
-            entry.cached(cacheCtx.local().entryEx(entry.key(), topVer));
+            entry.cached(cacheCtx.colocated().entryExx(entry.key(), topVer, true));
 
-        if (!remap && (cacheCtx.isNear() || cacheCtx.isLocal())) {
+        if (!remap && cacheCtx.isNear()) {
             if (entry.explicitVersion() == null) {
                 if (keyLockFut == null) {
                     keyLockFut = new KeyLockFuture();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index a0662647257..bfa1c7f9227 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -102,7 +102,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         if (tx.remainingTime() == -1)
             return false;
 
-        if ((entry.context().isNear() || entry.context().isLocal()) &&
+        if (entry.context().isNear() &&
             owner != null && tx.hasWriteKey(entry.txKey())) {
             if (keyLockFut != null)
                 keyLockFut.onKeyLocked(entry.txKey());
@@ -436,7 +436,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
                 ClusterNode primary = updated.primary();
 
-                assert !primary.isLocal() || !cctx.kernalContext().clientNode() || write.context().isLocal();
+                assert !primary.isLocal() || !cctx.kernalContext().clientNode();
 
                 // Minor optimization to not create MappingKey: on client node can not have mapping for local node.
                 Object key = cctx.kernalContext().clientNode() ? primary.id() :
@@ -633,9 +633,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         if (cached0.isDht())
             nodes = cacheCtx.topology().nodes(cached0.partition(), topVer);
         else
-            nodes = cacheCtx.isLocal() ?
-                cacheCtx.affinity().nodesByKey(entry.key(), topVer) :
-                cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer);
+            nodes = cacheCtx.topology().nodes(cacheCtx.affinity().partition(entry.key()), topVer);
 
         if (F.isEmpty(nodes)) {
             ClusterTopologyServerNotFoundException e = new ClusterTopologyServerNotFoundException("Failed to map " +
@@ -662,12 +660,10 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         // Must re-initialize cached entry while holding topology lock.
         if (cacheCtx.isNear())
             entry.cached(cacheCtx.nearTx().entryExx(entry.key(), topVer));
-        else if (!cacheCtx.isLocal())
-            entry.cached(cacheCtx.colocated().entryExx(entry.key(), topVer, true));
         else
-            entry.cached(cacheCtx.local().entryEx(entry.key(), topVer));
+            entry.cached(cacheCtx.colocated().entryExx(entry.key(), topVer, true));
 
-        if (cacheCtx.isNear() || cacheCtx.isLocal()) {
+        if (cacheCtx.isNear()) {
             if (entry.explicitVersion() == null && !remap) {
                 if (keyLockFut == null) {
                     keyLockFut = new KeyLockFuture();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
index c0fbfc23538..565247e7664 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
@@ -66,7 +66,7 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
         if (tx.timeout() > 0) {
             // Init keyLockFut to make sure it is created when {@link #onNearTxLocalTimeout} is called.
             for (IgniteTxEntry e : tx.writeEntries()) {
-                if (e.context().isNear() || e.context().isLocal()) {
+                if (e.context().isNear()) {
                     keyLockFut = new KeyLockFuture();
                     break;
                 }
@@ -74,7 +74,7 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
 
             if (tx.serializable() && keyLockFut == null) {
                 for (IgniteTxEntry e : tx.readEntries()) {
-                    if (e.context().isNear() || e.context().isLocal()) {
+                    if (e.context().isNear()) {
                         keyLockFut = new KeyLockFuture();
                         break;
                     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 9f402220d89..336c3eca340 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -323,13 +323,8 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
 
                 List<ClusterNode> nodes;
 
-                if (!cacheCtx.isLocal()) {
-                    GridDhtPartitionTopology top = cacheCtx.topology();
-
-                    nodes = top.nodes(cacheCtx.affinity().partition(txEntry.key()), topVer);
-                }
-                else
-                    nodes = cacheCtx.affinity().nodesByKey(txEntry.key(), topVer);
+                GridDhtPartitionTopology top = cacheCtx.topology();
+                nodes = top.nodes(cacheCtx.affinity().partition(txEntry.key()), topVer);
 
                 if (F.isEmpty(nodes)) {
                     onDone(new ClusterTopologyServerNotFoundException("Failed to map keys to nodes (partition " +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 102f2854ae2..659daa6c3ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -102,7 +102,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.C2;
 import org.apache.ignite.internal.util.typedef.CI1;
-import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.CX1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -118,7 +117,6 @@ import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionState;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.CREATE;
@@ -127,7 +125,6 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.NOO
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
-import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.SER_READ_EMPTY_ENTRY_VER;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry.SER_READ_NOT_EMPTY_VER;
 import static org.apache.ignite.internal.processors.tracing.MTC.TraceSurroundings;
 import static org.apache.ignite.internal.processors.tracing.SpanType.TX_NEAR_ENLIST_READ;
@@ -2436,26 +2433,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
                             }
                         }
 
-                        if (!missed.isEmpty() && cacheCtx.isLocal()) {
-                            AffinityTopologyVersion topVer = topologyVersionSnapshot();
-
-                            if (topVer == null)
-                                topVer = entryTopVer;
-
-                            return checkMissed(cacheCtx,
-                                topVer != null ? topVer : topologyVersion(),
-                                retMap,
-                                missed,
-                                deserializeBinary,
-                                skipVals,
-                                keepCacheObjects,
-                                skipStore,
-                                recovery,
-                                readRepairStrategy,
-                                needVer,
-                                expiryPlc0);
-                        }
-
                         if (readRepairStrategy != null) { // Checking and repairing each locked entry (if necessary).
                             // Providing the guarantee that all copies are updated when read repair operation is finished.
                             syncMode(FULL_SYNC);
@@ -3192,7 +3169,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
                     }
                 });
         }
-        else if (cacheCtx.isColocated()) {
+        // cacheCtx.isColocated() == true
+        else {
             if (readRepairStrategy != null) {
                 return new GridNearReadRepairCheckOnlyFuture(
                     topVer,
@@ -3298,169 +3276,6 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
                 });
             }
         }
-        else {
-            assert cacheCtx.isLocal();
-
-            return localCacheLoadMissing(cacheCtx,
-                topVer,
-                readThrough,
-                async,
-                keys,
-                skipVals,
-                needVer,
-                keepBinary,
-                recovery,
-                expiryPlc,
-                c);
-        }
-    }
-
-    /**
-     * @param cacheCtx Cache context.
-     * @param readThrough Read through flag.
-     * @param async if {@code True}, then loading will happen in a separate thread.
-     * @param keys Keys.
-     * @param skipVals Skip values flag.
-     * @param needVer If {@code true} version is required for loaded values.
-     * @param c Closure to be applied for loaded values.
-     * @param expiryPlc Expiry policy.
-     * @return Future with {@code True} value if loading took place.
-     */
-    private IgniteInternalFuture<Void> localCacheLoadMissing(
-        final GridCacheContext cacheCtx,
-        final AffinityTopologyVersion topVer,
-        final boolean readThrough,
-        boolean async,
-        final Collection<KeyCacheObject> keys,
-        boolean skipVals,
-        boolean needVer,
-        boolean keepBinary,
-        boolean recovery,
-        final ExpiryPolicy expiryPlc,
-        final GridInClosure3<KeyCacheObject, Object, GridCacheVersion> c
-    ) {
-        assert cacheCtx.isLocal() : cacheCtx.name();
-
-        if (!readThrough || !cacheCtx.readThrough()) {
-            for (KeyCacheObject key : keys)
-                c.apply(key, null, SER_READ_EMPTY_ENTRY_VER);
-
-            return new GridFinishedFuture<>();
-        }
-
-        try {
-            IgniteCacheExpiryPolicy expiryPlc0 = optimistic() ?
-                accessPolicy(cacheCtx, keys) :
-                cacheCtx.cache().expiryPolicy(expiryPlc);
-
-            Map<KeyCacheObject, GridCacheVersion> misses = null;
-
-            for (KeyCacheObject key : keys) {
-                while (true) {
-                    IgniteTxEntry txEntry = entry(cacheCtx.txKey(key));
-
-                    GridCacheEntryEx entry = txEntry == null ? cacheCtx.cache().entryEx(key) :
-                        txEntry.cached();
-
-                    if (entry == null)
-                        continue;
-
-                    try {
-                        EntryGetResult res = entry.innerGetVersioned(
-                            null,
-                            this,
-                            /*update-metrics*/!skipVals,
-                            /*event*/!skipVals,
-                            null,
-                            resolveTaskName(),
-                            expiryPlc0,
-                            txEntry == null ? keepBinary : txEntry.keepBinary(),
-                            null);
-
-                        if (res == null) {
-                            if (misses == null)
-                                misses = new LinkedHashMap<>();
-
-                            misses.put(key, entry.version());
-                        }
-                        else
-                            c.apply(key, skipVals ? true : res.value(), res.version());
-
-                        break;
-                    }
-                    catch (GridCacheEntryRemovedException ignore) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry, will retry: " + key);
-
-                        if (txEntry != null)
-                            txEntry.cached(cacheCtx.cache().entryEx(key, topologyVersion()));
-                    }
-                }
-            }
-
-            if (misses != null) {
-                final Map<KeyCacheObject, GridCacheVersion> misses0 = misses;
-
-                cacheCtx.store().loadAll(this, misses.keySet(), new CI2<KeyCacheObject, Object>() {
-                    @Override public void apply(KeyCacheObject key, Object val) {
-                        GridCacheVersion ver = misses0.remove(key);
-
-                        assert ver != null : key;
-
-                        if (val != null) {
-                            CacheObject cacheVal = cacheCtx.toCacheObject(val);
-
-                            while (true) {
-                                GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
-
-                                try {
-                                    cacheCtx.shared().database().ensureFreeSpace(cacheCtx.dataRegion());
-
-                                    EntryGetResult verVal = entry.versionedValue(cacheVal,
-                                        ver,
-                                        null,
-                                        null,
-                                        null);
-
-                                    if (log.isDebugEnabled()) {
-                                        log.debug("Set value loaded from store into entry [" +
-                                            "oldVer=" + ver +
-                                            ", newVer=" + verVal.version() +
-                                            ", entry=" + entry + ']');
-                                    }
-
-                                    ver = verVal.version();
-
-                                    break;
-                                }
-                                catch (GridCacheEntryRemovedException ignore) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Got removed entry, (will retry): " + entry);
-                                }
-                                catch (IgniteCheckedException e) {
-                                    // Wrap errors (will be unwrapped).
-                                    throw new GridClosureException(e);
-                                }
-                            }
-                        }
-                        else
-                            ver = SER_READ_EMPTY_ENTRY_VER;
-
-                        c.apply(key, val, ver);
-                    }
-                });
-
-                for (KeyCacheObject key : misses0.keySet())
-                    c.apply(key, null, SER_READ_EMPTY_ENTRY_VER);
-            }
-
-            return new GridFinishedFuture<>();
-        }
-        catch (IgniteCheckedException e) {
-            setRollbackOnly();
-
-            return new GridFinishedFuture<>(e);
-        }
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
deleted file mode 100644
index 7e91fbcabe4..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ /dev/null
@@ -1,263 +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.internal.processors.cache.local;
-
-import java.io.Externalizable;
-import java.util.Collection;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.GridCacheLocalConcurrentMap;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntryFactory;
-import org.apache.ignite.internal.processors.cache.GridCachePreloader;
-import org.apache.ignite.internal.processors.cache.GridCachePreloaderAdapter;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.transactions.TransactionIsolation;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Local cache implementation.
- */
-public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private GridCachePreloader preldr;
-
-    /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public GridLocalCache() {
-        // No-op.
-    }
-
-    /**
-     * @param ctx Cache registry.
-     */
-    public GridLocalCache(GridCacheContext<K, V> ctx) {
-        super(ctx);
-
-        preldr = new GridCachePreloaderAdapter(ctx.group());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteCheckedException {
-        if (map == null)
-            map = new GridCacheLocalConcurrentMap(ctx, entryFactory(), DFLT_START_CACHE_SIZE);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isLocal() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCachePreloader preloader() {
-        return preldr;
-    }
-
-    /**
-     * @return Entry factory.
-     */
-    private GridCacheMapEntryFactory entryFactory() {
-        return new GridCacheMapEntryFactory() {
-            @Override public GridCacheMapEntry create(
-                GridCacheContext ctx,
-                AffinityTopologyVersion topVer,
-                KeyCacheObject key
-            ) {
-                return new GridLocalCacheEntry(ctx, key);
-            }
-        };
-    }
-
-    /**
-     * @param key Key of entry.
-     * @return Cache entry.
-     */
-    @Nullable private GridLocalCacheEntry peekExx(KeyCacheObject key) {
-        return (GridLocalCacheEntry)peekEx(key);
-    }
-
-    /**
-     * @param key Key of entry.
-     * @return Cache entry.
-     */
-    GridLocalCacheEntry entryExx(KeyCacheObject key) {
-        return (GridLocalCacheEntry)entryEx(key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> txLockAsync(Collection<KeyCacheObject> keys,
-        long timeout,
-        IgniteTxLocalEx tx,
-        boolean isRead,
-        boolean retval,
-        TransactionIsolation isolation,
-        boolean invalidate,
-        long createTtl,
-        long accessTtl) {
-        return lockAllAsync(keys, timeout, tx, CU.empty0());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout) {
-        IgniteTxLocalEx tx = ctx.tm().localTx();
-
-        return lockAllAsync(ctx.cacheKeysView(keys), timeout, tx, CU.empty0());
-    }
-
-    /**
-     * @param keys Keys.
-     * @param timeout Timeout.
-     * @param tx Transaction.
-     * @param filter Filter.
-     * @return Future.
-     */
-    public IgniteInternalFuture<Boolean> lockAllAsync(Collection<KeyCacheObject> keys,
-        long timeout,
-        @Nullable IgniteTxLocalEx tx,
-        CacheEntryPredicate[] filter) {
-        if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(true);
-
-        GridLocalLockFuture<K, V> fut = new GridLocalLockFuture<>(ctx, keys, tx, this, timeout, filter);
-
-        try {
-            if (!fut.addEntries(keys))
-                return fut;
-
-            if (!ctx.mvcc().addFuture(fut))
-                fut.onError(new IgniteCheckedException("Duplicate future ID (internal error): " + fut));
-
-            // Must have future added prior to checking locks.
-            fut.checkLocks();
-
-            return fut;
-        }
-        catch (IgniteCheckedException e) {
-            fut.onError(e);
-
-            return fut;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void unlockAll(
-        Collection<? extends K> keys
-    ) throws IgniteCheckedException {
-        AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
-
-        for (K key : keys) {
-            GridLocalCacheEntry entry = peekExx(ctx.toCacheKeyObject(key));
-
-            if (entry != null && ctx.isAll(entry, CU.empty0())) {
-                entry.releaseLocal();
-
-                entry.touch();
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> removeAllAsync() {
-        return ctx.closures().callLocalSafe(new GridPlainCallable<Void>() {
-            @Override public Void call() throws Exception {
-                removeAll();
-
-                return null;
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeferredDelete(GridCacheEntryEx entry, GridCacheVersion ver) {
-        assert false : "Should not be called";
-    }
-
-    /**
-     * @param fut Clears future from cache.
-     */
-    void onFutureDone(GridLocalLockFuture fut) {
-        if (ctx.mvcc().removeVersionedFuture(fut)) {
-            if (log().isDebugEnabled())
-                log().debug("Explicitly removed future from map of futures: " + fut);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public long localSizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException {
-        PeekModes modes = parsePeekModes(peekModes, true);
-
-        modes.primary = true;
-        modes.backup = true;
-
-        if (modes.offheap)
-            return ctx.offheap().cacheEntriesCount(ctx.cacheId());
-        else if (modes.heap)
-            return size();
-        else
-            return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long localSizeLong(int part, CachePeekMode[] peekModes) throws IgniteCheckedException {
-        return localSizeLong(peekModes);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void preloadPartition(int part) throws IgniteCheckedException {
-        ctx.offheap().preloadPartition(part);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> preloadPartitionAsync(int part) throws IgniteCheckedException {
-        return ctx.closures().callLocalSafe(new GridPlainCallable<Void>() {
-            @Override public Void call() throws Exception {
-                preloadPartition(part);
-
-                return null;
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean localPreloadPartition(int part) throws IgniteCheckedException {
-        ctx.offheap().preloadPartition(part);
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion nextVersion() {
-        return ctx.versions().next(ctx.shared().kernalContext().discovery().topologyVersion());
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
deleted file mode 100644
index f7167595f6d..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ /dev/null
@@ -1,382 +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.internal.processors.cache.local;
-
-import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
-import org.apache.ignite.internal.processors.cache.GridCacheMvcc;
-import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Cache entry for local caches.
- */
-@SuppressWarnings({"TooBroadScope"})
-public class GridLocalCacheEntry extends GridCacheMapEntry {
-    /**
-     * @param ctx  Cache registry.
-     * @param key  Cache key.
-     */
-    GridLocalCacheEntry(
-        GridCacheContext ctx,
-        KeyCacheObject key
-    ) {
-        super(ctx, key);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isLocal() {
-        return true;
-    }
-
-    /**
-     * Add local candidate.
-     *
-     * @param threadId Owning thread ID.
-     * @param ver Lock version.
-     * @param serOrder Version for serializable transactions ordering.
-     * @param serReadVer Optional read entry version for optimistic serializable transaction.
-     * @param timeout Timeout to acquire lock.
-     * @param reenter Reentry flag.
-     * @param tx Transaction flag.
-     * @param implicitSingle Implicit transaction flag.
-     * @param read Read lock flag.
-     * @return New candidate.
-     * @throws GridCacheEntryRemovedException If entry has been removed.
-     */
-    @Nullable GridCacheMvccCandidate addLocal(
-        long threadId,
-        GridCacheVersion ver,
-        @Nullable GridCacheVersion serOrder,
-        @Nullable GridCacheVersion serReadVer,
-        long timeout,
-        boolean reenter,
-        boolean tx,
-        boolean implicitSingle,
-        boolean read
-    ) throws GridCacheEntryRemovedException {
-        assert serReadVer == null || serOrder != null;
-
-        CacheObject val;
-        GridCacheMvccCandidate cand;
-        CacheLockCandidates prev;
-        CacheLockCandidates owner = null;
-
-        lockEntry();
-
-        try {
-            checkObsolete();
-
-            if (serReadVer != null) {
-                if (!checkSerializableReadVersion(serReadVer))
-                    return null;
-            }
-
-            GridCacheMvcc mvcc = mvccExtras();
-
-            if (mvcc == null) {
-                mvcc = new GridCacheMvcc(cctx);
-
-                mvccExtras(mvcc);
-            }
-
-            prev = mvcc.localOwners();
-
-            cand = mvcc.addLocal(
-                this,
-                /*nearNodeId*/null,
-                /*nearVer*/null,
-                threadId,
-                ver,
-                timeout,
-                serOrder,
-                reenter,
-                tx,
-                implicitSingle,
-                /*dht-local*/false,
-                read
-            );
-
-            if (mvcc.isEmpty())
-                mvccExtras(null);
-            else
-                owner = mvcc.localOwners();
-
-            val = this.val;
-        }
-        finally {
-            unlockEntry();
-        }
-
-        if (cand != null && !cand.reentry())
-            cctx.mvcc().addNext(cctx, cand);
-
-        checkOwnerChanged(prev, owner, val);
-
-        return cand;
-    }
-
-    /**
-     * @param cand Candidate.
-     */
-    void readyLocal(GridCacheMvccCandidate cand) {
-        CacheObject val;
-        CacheLockCandidates prev = null;
-        CacheLockCandidates owner = null;
-
-        lockEntry();
-
-        try {
-            GridCacheMvcc mvcc = mvccExtras();
-
-            if (mvcc != null) {
-                prev = mvcc.localOwners();
-
-                owner = mvcc.readyLocal(cand);
-
-                if (mvcc.isEmpty())
-                    mvccExtras(null);
-            }
-
-            val = this.val;
-        }
-        finally {
-            unlockEntry();
-        }
-
-        checkOwnerChanged(prev, owner, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean tmLock(IgniteInternalTx tx,
-        long timeout,
-        @Nullable GridCacheVersion serOrder,
-        GridCacheVersion serReadVer,
-        boolean read)
-        throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate cand = addLocal(
-            tx.threadId(),
-            tx.xidVersion(),
-            serOrder,
-            serReadVer,
-            timeout,
-            /*reenter*/false,
-            /*tx*/true,
-            tx.implicitSingle(),
-            read
-        );
-
-        if (cand != null) {
-            readyLocal(cand);
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * Rechecks if lock should be reassigned.
-     *
-     * @param ver Thread chain version.
-     *
-     * @return {@code True} if thread chain processing must be stopped.
-     */
-    public boolean recheck(GridCacheVersion ver) {
-        CacheObject val;
-        CacheLockCandidates prev = null;
-        CacheLockCandidates owner = null;
-
-        lockEntry();
-
-        try {
-            GridCacheMvcc mvcc = mvccExtras();
-
-            if (mvcc != null) {
-                prev = mvcc.allOwners();
-
-                owner = mvcc.recheck();
-
-                if (mvcc.isEmpty())
-                    mvccExtras(null);
-            }
-
-            val = this.val;
-        }
-        finally {
-            unlockEntry();
-        }
-
-        boolean lockedByThreadChainVer = owner != null && owner.hasCandidate(ver);
-
-        // If locked by the thread chain version no need to do recursive thread chain scans for the same chain.
-        // This call must be made outside of synchronization.
-        checkOwnerChanged(prev, owner, val, lockedByThreadChainVer);
-
-        return !lockedByThreadChainVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkThreadChain(GridCacheMvccCandidate owner) {
-        assert !lockedByCurrentThread();
-
-        assert owner != null;
-        assert owner.owner() || owner.used() : "Neither owner or used flags are set on ready local candidate: " +
-            owner;
-
-        if (owner.next() != null) {
-            for (GridCacheMvccCandidate cand = owner.next(); cand != null; cand = cand.next()) {
-                assert cand.local();
-
-                // Allow next lock in the thread to proceed.
-                if (!cand.used()) {
-                    GridCacheContext cctx0 = cand.parent().context();
-
-                    GridLocalCacheEntry e = (GridLocalCacheEntry)cctx0.cache().peekEx(cand.parent().key());
-
-                    // At this point candidate may have been removed and entry destroyed, so we check for null.
-                    if (e == null || e.recheck(owner.version()))
-                        break;
-                }
-            }
-        }
-    }
-
-    /**
-     * Releases local lock.
-     */
-    void releaseLocal() {
-        releaseLocal(Thread.currentThread().getId());
-    }
-
-    /**
-     * Releases local lock.
-     *
-     * @param threadId Thread ID.
-     */
-    private void releaseLocal(long threadId) {
-        CacheObject val;
-        CacheLockCandidates prev = null;
-        CacheLockCandidates owner = null;
-
-        lockEntry();
-
-        try {
-            GridCacheMvcc mvcc = mvccExtras();
-
-            if (mvcc != null) {
-                prev = mvcc.localOwners();
-
-                mvcc.releaseLocal(threadId);
-
-                if (mvcc.isEmpty())
-                    mvccExtras(null);
-                else
-                    owner = mvcc.allOwners();
-            }
-
-            val = this.val;
-        }
-        finally {
-            unlockEntry();
-        }
-
-        if (prev != null) {
-            for (int i = 0; i < prev.size(); i++) {
-                GridCacheMvccCandidate cand = prev.candidate(i);
-
-                boolean unlocked = owner == null || !owner.hasCandidate(cand.version());
-
-                if (unlocked)
-                    checkThreadChain(cand);
-            }
-        }
-
-        checkOwnerChanged(prev, owner, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean removeLock(GridCacheVersion ver) throws GridCacheEntryRemovedException {
-        CacheObject val;
-        CacheLockCandidates prev = null;
-        CacheLockCandidates owner = null;
-
-        GridCacheMvccCandidate doomed;
-
-        GridCacheVersion deferredDelVer;
-
-        lockEntry();
-
-        try {
-            GridCacheVersion obsoleteVer = obsoleteVersionExtras();
-
-            if (obsoleteVer != null && !obsoleteVer.equals(ver))
-                checkObsolete();
-
-            GridCacheMvcc mvcc = mvccExtras();
-
-            doomed = mvcc == null ? null : mvcc.candidate(ver);
-
-            if (doomed != null) {
-                prev = mvcc.allOwners();
-
-                mvcc.remove(ver);
-
-                if (mvcc.isEmpty())
-                    mvccExtras(null);
-                else
-                    owner = mvcc.allOwners();
-            }
-
-            val = this.val;
-
-            deferredDelVer = this.ver;
-        }
-        finally {
-            unlockEntry();
-        }
-
-        if (val == null) {
-            boolean deferred = cctx.deferredDelete() && !detached() && !isInternal();
-
-            if (deferred) {
-                if (deferredDelVer != null)
-                    cctx.onDeferredDelete(this, deferredDelVer);
-            }
-        }
-
-        if (doomed != null)
-            checkThreadChain(doomed);
-
-        checkOwnerChanged(prev, owner, val);
-
-        return doomed != null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return toStringWithTryLock(() -> S.toString(GridLocalCacheEntry.class, this, super.toString()));
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockCallback.java
deleted file mode 100644
index 47b864c38ee..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockCallback.java
+++ /dev/null
@@ -1,39 +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.internal.processors.cache.local;
-
-import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
-
-/**
- * @param <K> Key type.
- * @param <V> Value type.
- */
-interface GridLocalLockCallback {
-    /**
-     * Called when entry lock ownership changes. This call
-     * happens outside of synchronization so external callbacks
-     * can be made from this call.
-     *
-     * @param entry Entry whose owner has changed.
-     * @param prev Previous candidate.
-     * @param owner Current candidate.
-     */
-    public void onOwnerChanged(GridLocalCacheEntry entry,
-        GridCacheMvccCandidate prev,
-        GridCacheMvccCandidate owner);
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
deleted file mode 100644
index 372ebeac45f..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
+++ /dev/null
@@ -1,556 +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.internal.processors.cache.local;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheFutureAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
-import org.apache.ignite.internal.processors.cache.GridCacheVersionedFuture;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
-import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
-import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
-import org.apache.ignite.internal.util.tostring.GridToStringExclude;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.transactions.TransactionDeadlockException;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Cache lock future.
- */
-public final class GridLocalLockFuture<K, V> extends GridCacheFutureAdapter<Boolean>
-    implements GridCacheVersionedFuture<Boolean> {
-    /** Logger reference. */
-    private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
-
-    /** Error updater. */
-    private static final AtomicReferenceFieldUpdater<GridLocalLockFuture, Throwable> ERR_UPD =
-        AtomicReferenceFieldUpdater.newUpdater(GridLocalLockFuture.class, Throwable.class, "err");
-
-    /** Logger. */
-    private static IgniteLogger log;
-
-    /** Cache registry. */
-    @GridToStringExclude
-    private GridCacheContext<K, V> cctx;
-
-    /** Underlying cache. */
-    @GridToStringExclude
-    private GridLocalCache<K, V> cache;
-
-    /** Lock owner thread. */
-    @GridToStringInclude
-    private long threadId;
-
-    /**
-     * Keys locked so far.
-     *
-     * Thread created this object iterates over entries and tries to lock each of them.
-     * If it finds some entry already locked by another thread it registers callback which will be executed
-     * by the thread owning the lock.
-     *
-     * Thus access to this collection must be synchronized except cases
-     * when this object is yet local to the thread created it.
-     */
-    @GridToStringExclude
-    private List<GridLocalCacheEntry> entries;
-
-    /** Future ID. */
-    private IgniteUuid futId;
-
-    /** Lock version. */
-    private GridCacheVersion lockVer;
-
-    /** Error. */
-    private volatile Throwable err;
-
-    /** Timeout object. */
-    @GridToStringExclude
-    private LockTimeoutObject timeoutObj;
-
-    /** Lock timeout. */
-    private final long timeout;
-
-    /** Filter. */
-    private CacheEntryPredicate[] filter;
-
-    /** Transaction. */
-    private IgniteTxLocalEx tx;
-
-    /** Trackable flag. */
-    private boolean trackable = true;
-
-    /**
-     * @param cctx Registry.
-     * @param keys Keys to lock.
-     * @param tx Transaction.
-     * @param cache Underlying cache.
-     * @param timeout Lock acquisition timeout.
-     * @param filter Filter.
-     */
-    GridLocalLockFuture(
-        GridCacheContext<K, V> cctx,
-        Collection<KeyCacheObject> keys,
-        IgniteTxLocalEx tx,
-        GridLocalCache<K, V> cache,
-        long timeout,
-        CacheEntryPredicate[] filter) {
-        assert keys != null;
-        assert cache != null;
-        assert (tx != null && timeout >= 0) || tx == null;
-
-        this.cctx = cctx;
-        this.cache = cache;
-        this.timeout = timeout;
-        this.filter = filter;
-        this.tx = tx;
-
-        ignoreInterrupts();
-
-        threadId = tx == null ? Thread.currentThread().getId() : tx.threadId();
-
-        lockVer = tx != null ? tx.xidVersion() : cctx.cache().nextVersion();
-
-        futId = IgniteUuid.randomUuid();
-
-        entries = new ArrayList<>(keys.size());
-
-        if (log == null)
-            log = U.logger(cctx.kernalContext(), logRef, GridLocalLockFuture.class);
-
-        if (tx != null && tx instanceof GridNearTxLocal && !((GridNearTxLocal)tx).updateLockFuture(null, this)) {
-            GridNearTxLocal tx0 = (GridNearTxLocal)tx;
-
-            onError(tx0.timedOut() ? tx0.timeoutException() : tx0.rollbackException());
-        }
-    }
-
-    /**
-     * @param keys Keys.
-     * @return {@code False} in case of error.
-     * @throws IgniteCheckedException If failed.
-     */
-    public boolean addEntries(Collection<KeyCacheObject> keys) throws IgniteCheckedException {
-        for (KeyCacheObject key : keys) {
-            while (true) {
-                GridLocalCacheEntry entry = null;
-
-                try {
-                    entry = cache.entryExx(key);
-
-                    entry.unswap(false);
-
-                    if (!cctx.isAll(entry, filter)) {
-                        onFailed();
-
-                        return false;
-                    }
-
-                    // Removed exception may be thrown here.
-                    GridCacheMvccCandidate cand = addEntry(entry);
-
-                    if (cand == null && isDone())
-                        return false;
-
-                    break;
-                }
-                catch (GridCacheEntryRemovedException ignored) {
-                    if (log.isDebugEnabled())
-                        log.debug("Got removed entry in lockAsync(..) method (will retry): " + entry);
-                }
-            }
-        }
-
-        if (timeout > 0) {
-            timeoutObj = new LockTimeoutObject();
-
-            cctx.time().addTimeoutObject(timeoutObj);
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid futureId() {
-        return futId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCacheVersion version() {
-        return lockVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onNodeLeft(UUID nodeId) {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean trackable() {
-        return trackable;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void markNotTrackable() {
-        trackable = false;
-    }
-
-    /**
-     * @return Entries.
-     */
-    private List<GridLocalCacheEntry> entries() {
-        return entries;
-    }
-
-    /**
-     * @return {@code True} if transaction is not {@code null}.
-     */
-    private boolean inTx() {
-        return tx != null;
-    }
-
-    /**
-     * @return {@code True} if implicit transaction.
-     */
-    private boolean implicitSingle() {
-        return tx != null && tx.implicitSingle();
-    }
-
-    /**
-     * @param cached Entry.
-     * @return {@code True} if locked.
-     * @throws GridCacheEntryRemovedException If removed.
-     */
-    private boolean locked(GridCacheEntryEx cached) throws GridCacheEntryRemovedException {
-        // Reentry-aware check.
-        return (cached.lockedLocally(lockVer) || (cached.lockedByThread(threadId))) &&
-            filter(cached); // If filter failed, lock is failed.
-    }
-
-    /**
-     * Adds entry to future.
-     *
-     * @param entry Entry to add.
-     * @return Lock candidate.
-     * @throws GridCacheEntryRemovedException If entry was removed.
-     */
-    @Nullable private GridCacheMvccCandidate addEntry(GridLocalCacheEntry entry)
-        throws GridCacheEntryRemovedException {
-        // Add local lock first, as it may throw GridCacheEntryRemovedException.
-        GridCacheMvccCandidate c = entry.addLocal(
-            threadId,
-            lockVer,
-            null,
-            null,
-            timeout,
-            !inTx(),
-            inTx(),
-            implicitSingle(),
-            false
-        );
-
-        entries.add(entry);
-
-        if (c == null && timeout < 0) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to acquire lock with negative timeout: " + entry);
-
-            onFailed();
-
-            return null;
-        }
-
-        if (c != null) {
-            // Immediately set lock to ready.
-            entry.readyLocal(c);
-        }
-
-        return c;
-    }
-
-    /**
-     * Undoes all locks.
-     */
-    private void undoLocks() {
-        Collection<GridLocalCacheEntry> entriesCp = entriesCopy();
-
-        for (GridLocalCacheEntry e : entriesCp) {
-            try {
-                e.removeLock(lockVer);
-            }
-            catch (GridCacheEntryRemovedException ignore) {
-                if (log.isDebugEnabled())
-                    log.debug("Got removed entry while undoing locks: " + e);
-            }
-        }
-    }
-
-    /**
-     * Need of synchronization here is explained in the field's {@link GridLocalLockFuture#entries} comment.
-     *
-     * @return Copy of entries collection.
-     */
-    private synchronized Collection<GridLocalCacheEntry> entriesCopy() {
-        return new ArrayList<>(entries());
-    }
-
-    /**
-     *
-     */
-    void onFailed() {
-        undoLocks();
-
-        onComplete(false);
-    }
-
-    /**
-     * @param t Error.
-     */
-    void onError(Throwable t) {
-        if (ERR_UPD.compareAndSet(this, null, t))
-            onFailed();
-    }
-
-    /**
-     * @param cached Entry to check.
-     * @return {@code True} if filter passed.
-     */
-    private boolean filter(GridCacheEntryEx cached) {
-        try {
-            if (!cctx.isAll(cached, filter)) {
-                if (log.isDebugEnabled())
-                    log.debug("Filter didn't pass for entry (will fail lock): " + cached);
-
-                onFailed();
-
-                return false;
-            }
-
-            return true;
-        }
-        catch (IgniteCheckedException e) {
-            onError(e);
-
-            return false;
-        }
-    }
-
-    /**
-     * Explicitly check if lock was acquired.
-     */
-    void checkLocks() {
-        if (!isDone()) {
-            for (int i = 0; i < entries.size(); i++) {
-                while (true) {
-                    GridCacheEntryEx cached = entries.get(i);
-
-                    try {
-                        if (!locked(cached))
-                            return;
-
-                        break;
-                    }
-                    // Possible in concurrent cases, when owner is changed after locks
-                    // have been released or cancelled.
-                    catch (GridCacheEntryRemovedException ignore) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry in onOwnerChanged method (will retry): " + cached);
-
-                        // Replace old entry with new one.
-                        entries.add(i, (GridLocalCacheEntry)cache.entryEx(cached.key()));
-                    }
-                }
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("Local lock acquired for entries: " + entries);
-
-            onComplete(true);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner) {
-        if (!isDone()) {
-            for (int i = 0; i < entries.size(); i++) {
-                while (true) {
-                    GridCacheEntryEx cached = entries.get(i);
-
-                    try {
-                        if (!locked(cached))
-                            return true;
-
-                        break;
-                    }
-                    // Possible in concurrent cases, when owner is changed after locks
-                    // have been released or cancelled.
-                    catch (GridCacheEntryRemovedException ignore) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry in onOwnerChanged method (will retry): " + cached);
-
-                        // Replace old entry with new one.
-                        entries.add(i, (GridLocalCacheEntry)cache.entryEx(cached.key()));
-                    }
-                }
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("Local lock acquired for entries: " + entries);
-
-            onComplete(true);
-        }
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean cancel() {
-        if (onCancelled()) {
-            // Remove all locks.
-            undoLocks();
-
-            onComplete(false);
-        }
-
-        return isCancelled();
-    }
-
-    /**
-     * Completeness callback.
-     *
-     * @param success If {@code true}, then lock has been acquired.
-     */
-    private void onComplete(boolean success) {
-        if (!success)
-            undoLocks();
-
-        if (tx != null && success)
-            ((GridNearTxLocal)tx).clearLockFuture(this);
-
-        if (onDone(success, err)) {
-            if (log.isDebugEnabled())
-                log.debug("Completing future: " + this);
-
-            cache.onFutureDone(this);
-
-            if (timeoutObj != null)
-                cctx.time().removeTimeoutObject(timeoutObj);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridLocalLockFuture.class, this);
-    }
-
-    /**
-     * Lock request timeout object.
-     */
-    private class LockTimeoutObject extends GridTimeoutObjectAdapter {
-        /**
-         * Default constructor.
-         */
-        LockTimeoutObject() {
-            super(timeout);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings({"ForLoopReplaceableByForEach"})
-        @Override public void onTimeout() {
-            if (log.isDebugEnabled())
-                log.debug("Timed out waiting for lock response: " + this);
-
-            if (inTx()) {
-                if (cctx.tm().deadlockDetectionEnabled()) {
-                    Set<IgniteTxKey> keys = new HashSet<>();
-
-                    List<GridLocalCacheEntry> entries = entries();
-
-                    for (int i = 0; i < entries.size(); i++) {
-                        GridLocalCacheEntry e = entries.get(i);
-
-                        List<GridCacheMvccCandidate> mvcc = e.mvccAllLocal();
-
-                        if (mvcc == null)
-                            continue;
-
-                        GridCacheMvccCandidate cand = mvcc.get(0);
-
-                        if (cand.owner() && cand.tx() && !cand.version().equals(tx.xidVersion()))
-                            keys.add(e.txKey());
-                    }
-
-                    IgniteInternalFuture<TxDeadlock> fut = cctx.tm().detectDeadlock(tx, keys);
-
-                    fut.listen(new IgniteInClosure<IgniteInternalFuture<TxDeadlock>>() {
-                        @Override public void apply(IgniteInternalFuture<TxDeadlock> fut) {
-                            try {
-                                TxDeadlock deadlock = fut.get();
-
-                                err = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided " +
-                                    "timeout for transaction [timeout=" + tx.timeout() + ", tx=" + CU.txString(tx) + ']',
-                                    deadlock != null ? new TransactionDeadlockException(deadlock.toString(cctx.shared())) :
-                                        null);
-                            }
-                            catch (IgniteCheckedException e) {
-                                err = e;
-
-                                U.warn(log, "Failed to detect deadlock.", e);
-                            }
-
-                            onComplete(false);
-                        }
-                    });
-                }
-                else
-                    err = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided " +
-                        "timeout for transaction [timeout=" + tx.timeout() + ", tx=" + CU.txString(tx) + ']');
-            }
-            else
-                onComplete(false);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(LockTimeoutObject.class, this);
-        }
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
deleted file mode 100644
index bbe7a8fb343..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ /dev/null
@@ -1,1580 +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.internal.processors.cache.local.atomic;
-
-import java.io.Externalizable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import javax.cache.expiry.ExpiryPolicy;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.EntryProcessorResult;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.ReadRepairStrategy;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
-import org.apache.ignite.internal.processors.cache.CacheInvokeEntry;
-import org.apache.ignite.internal.processors.cache.CacheInvokeResult;
-import org.apache.ignite.internal.processors.cache.CacheLazyEntry;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheOperationContext;
-import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
-import org.apache.ignite.internal.processors.cache.CacheStorePartialUpdateException;
-import org.apache.ignite.internal.processors.cache.EntryGetResult;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheOperation;
-import org.apache.ignite.internal.processors.cache.GridCachePreloader;
-import org.apache.ignite.internal.processors.cache.GridCachePreloaderAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheReturn;
-import org.apache.ignite.internal.processors.cache.IgniteCacheExpiryPolicy;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.LockedEntriesInfo;
-import org.apache.ignite.internal.processors.cache.local.GridLocalCache;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.resource.GridResourceIoc;
-import org.apache.ignite.internal.util.F0;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.lang.GridTuple3;
-import org.apache.ignite.internal.util.typedef.C1;
-import org.apache.ignite.internal.util.typedef.CI1;
-import org.apache.ignite.internal.util.typedef.CX1;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.plugin.security.SecurityPermission;
-import org.apache.ignite.thread.IgniteThread;
-import org.apache.ignite.transactions.TransactionIsolation;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
-import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
-import static org.apache.ignite.internal.processors.cache.GridCacheOperation.UPDATE;
-
-/**
- * Non-transactional local cache.
- */
-public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private GridCachePreloader preldr;
-
-    /** Locked entries info for each thread. */
-    private final LockedEntriesInfo lockedEntriesInfo = new LockedEntriesInfo();
-
-    /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public GridLocalAtomicCache() {
-        // No-op.
-    }
-
-    /**
-     * @param ctx Cache context.
-     */
-    public GridLocalAtomicCache(GridCacheContext<K, V> ctx) {
-        super(ctx);
-
-        preldr = new GridCachePreloaderAdapter(ctx.group());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkJta() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isLocal() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCachePreloader preloader() {
-        return preldr;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        return (V)updateAllInternal(UPDATE,
-            Collections.singleton(key),
-            Collections.singleton(val),
-            null,
-            expiryPerCall(),
-            true,
-            false,
-            filter,
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        Boolean res = (Boolean)updateAllInternal(UPDATE,
-            Collections.singleton(key),
-            Collections.singleton(val),
-            null,
-            expiryPerCall(),
-            false,
-            false,
-            filter,
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-
-        assert res != null;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<V> getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        return updateAllAsync0(F0.asMap(key, val),
-            null,
-            null,
-            true,
-            false,
-            filter);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        return updateAllAsync0(F0.asMap(key, val),
-            null,
-            null,
-            false,
-            false,
-            filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void putAll0(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        updateAllInternal(UPDATE,
-            m.keySet(),
-            m.values(),
-            null,
-            expiryPerCall(),
-            false,
-            false,
-            null,
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> putAllAsync0(Map<? extends K, ? extends V> m) {
-        return updateAllAsync0(m,
-            null,
-            null,
-            false,
-            false,
-            null).chain(RET2NULL);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected V getAndRemove0(K key) throws IgniteCheckedException {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        return (V)updateAllInternal(DELETE,
-            Collections.singleton(key),
-            null,
-            null,
-            expiryPerCall(),
-            true,
-            false,
-            null,
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<V> getAndRemoveAsync0(K key) {
-        return removeAllAsync0(Collections.singletonList(key), true, false, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeAll0(Collection<? extends K> keys) throws IgniteCheckedException {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        updateAllInternal(DELETE,
-            keys,
-            null,
-            null,
-            expiryPerCall(),
-            false,
-            false,
-            null,
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Object> removeAllAsync0(Collection<? extends K> keys) {
-        return removeAllAsync0(keys, false, false, null).chain(RET2NULL);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean remove0(K key, final CacheEntryPredicate filter) throws IgniteCheckedException {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        Boolean rmv = (Boolean)updateAllInternal(DELETE,
-            Collections.singleton(key),
-            null,
-            null,
-            expiryPerCall(),
-            false,
-            false,
-            filter,
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-
-        assert rmv != null;
-
-        return rmv;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Boolean> removeAsync0(K key, @Nullable CacheEntryPredicate filter) {
-        return removeAllAsync0(Collections.singletonList(key), false, false, filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> removeAllAsync() {
-        return ctx.closures().callLocalSafe(new GridPlainCallable<Void>() {
-            @Override public Void call() throws Exception {
-                removeAll();
-
-                return null;
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override protected V get(
-        final K key,
-        String taskName,
-        boolean deserializeBinary,
-        boolean needVer) throws IgniteCheckedException {
-        Map<K, V> m = getAllInternal(Collections.singleton(key),
-            ctx.readThrough(),
-            taskName,
-            deserializeBinary,
-            false,
-            needVer);
-
-        assert m.isEmpty() || m.size() == 1 : m.size();
-
-        return F.firstValue(m);
-    }
-
-    /** {@inheritDoc} */
-    @Override public final Map<K, V> getAll(
-        Collection<? extends K> keys,
-        boolean deserializeBinary,
-        boolean needVer,
-        boolean recovery,
-        ReadRepairStrategy readRepairStrategy) throws IgniteCheckedException {
-        A.notNull(keys, "keys");
-
-        String taskName = ctx.kernalContext().job().currentTaskName();
-
-        return getAllInternal(keys,
-            ctx.readThrough(),
-            taskName,
-            deserializeBinary,
-            false,
-            needVer);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
-        @Nullable final Collection<? extends K> keys,
-        final boolean forcePrimary,
-        boolean skipTx,
-        final String taskName,
-        final boolean deserializeBinary,
-        boolean recovery,
-        ReadRepairStrategy readRepairStrategy,
-        final boolean skipVals,
-        final boolean needVer
-    ) {
-        A.notNull(keys, "keys");
-
-        final boolean storeEnabled = ctx.readThrough();
-
-        return asyncOp(new GridPlainCallable<Map<K, V>>() {
-            @Override public Map<K, V> call() throws Exception {
-                return getAllInternal(keys, storeEnabled, taskName, deserializeBinary, skipVals, needVer);
-            }
-        });
-    }
-
-    /**
-     * Entry point to all public API get methods.
-     *
-     * @param keys Keys to remove.
-     * @param storeEnabled Store enabled flag.
-     * @param taskName Task name.
-     * @param deserializeBinary Deserialize binary .
-     * @param skipVals Skip value flag.
-     * @param needVer Need version.
-     * @return Key-value map.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    private Map<K, V> getAllInternal(@Nullable Collection<? extends K> keys,
-        boolean storeEnabled,
-        String taskName,
-        boolean deserializeBinary,
-        boolean skipVals,
-        boolean needVer
-    ) throws IgniteCheckedException {
-        ctx.checkSecurity(SecurityPermission.CACHE_READ);
-
-        if (F.isEmpty(keys))
-            return Collections.emptyMap();
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        Map<K, V> vals = U.newHashMap(keys.size());
-
-        warnIfUnordered(keys, BulkOperation.GET);
-
-        final IgniteCacheExpiryPolicy expiry = expiryPolicy(opCtx != null ? opCtx.expiry() : null);
-
-        boolean success = true;
-        boolean readNoEntry = ctx.readNoEntry(expiry, false);
-        final boolean evt = !skipVals;
-
-        ctx.shared().database().checkpointReadLock();
-
-        try {
-            for (K key : keys) {
-                if (key == null)
-                    throw new NullPointerException("Null key.");
-
-                KeyCacheObject cacheKey = ctx.toCacheKeyObject(key);
-
-                boolean skipEntry = readNoEntry;
-
-                if (readNoEntry) {
-                    CacheDataRow row = ctx.offheap().read(ctx, cacheKey);
-
-                    if (row != null) {
-                        long expireTime = row.expireTime();
-
-                        if (expireTime == 0 || expireTime > U.currentTimeMillis()) {
-                            ctx.addResult(vals,
-                                cacheKey,
-                                row.value(),
-                                skipVals,
-                                false,
-                                deserializeBinary,
-                                true,
-                                null,
-                                row.version(),
-                                0,
-                                0,
-                                needVer,
-                                null);
-
-                            if (ctx.statisticsEnabled() && !skipVals)
-                                metrics0().onRead(true);
-
-                            if (evt) {
-                                ctx.events().readEvent(cacheKey,
-                                    null,
-                                    null,
-                                    row.value(),
-                                    taskName,
-                                    !deserializeBinary);
-                            }
-                        }
-                        else
-                            skipEntry = false;
-                    }
-                    else
-                        success = false;
-                }
-
-                if (!skipEntry) {
-                    GridCacheEntryEx entry = null;
-
-                    while (true) {
-                        try {
-                            entry = entryEx(cacheKey);
-
-                            if (entry != null) {
-                                CacheObject v;
-
-                                if (needVer) {
-                                    EntryGetResult res = entry.innerGetVersioned(
-                                        null,
-                                        null,
-                                        /*update-metrics*/false,
-                                        /*event*/evt,
-                                        null,
-                                        taskName,
-                                        expiry,
-                                        !deserializeBinary,
-                                        null);
-
-                                    if (res != null) {
-                                        ctx.addResult(
-                                            vals,
-                                            cacheKey,
-                                            res,
-                                            skipVals,
-                                            false,
-                                            deserializeBinary,
-                                            true,
-                                            needVer);
-                                    }
-                                    else
-                                        success = false;
-                                }
-                                else {
-                                    v = entry.innerGet(
-                                        null,
-                                        null,
-                                        /*read-through*/false,
-                                        /*update-metrics*/true,
-                                        /*event*/evt,
-                                        null,
-                                        taskName,
-                                        expiry,
-                                        !deserializeBinary);
-
-                                    if (v != null) {
-                                        ctx.addResult(vals,
-                                            cacheKey,
-                                            v,
-                                            skipVals,
-                                            false,
-                                            deserializeBinary,
-                                            true,
-                                            null,
-                                            0,
-                                            0,
-                                            null);
-                                    }
-                                    else
-                                        success = false;
-                                }
-                            }
-
-                            break; // While.
-                        }
-                        catch (GridCacheEntryRemovedException ignored) {
-                            // No-op, retry.
-                        }
-                        finally {
-                            if (entry != null)
-                                entry.touch();
-                        }
-
-                        if (!success && storeEnabled)
-                            break;
-                    }
-                }
-                if (!success) {
-                    if (!storeEnabled && ctx.statisticsEnabled() && !skipVals)
-                        metrics0().onRead(false);
-                }
-            }
-        }
-        finally {
-            ctx.shared().database().checkpointReadUnlock();
-        }
-
-        if (success || !storeEnabled)
-            return vals;
-
-        return getAllAsync(
-            keys,
-            null,
-            opCtx == null || !opCtx.skipStore(),
-            false,
-            taskName,
-            deserializeBinary,
-            opCtx != null && opCtx.recovery(),
-            null,
-            /*force primary*/false,
-            expiry,
-            skipVals,
-            needVer).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> EntryProcessorResult<T> invoke(K key,
-        EntryProcessor<K, V, T> entryProcessor,
-        Object... args) throws IgniteCheckedException {
-        return invokeAsync(key, entryProcessor, args).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
-        final EntryProcessor<K, V, T> entryProcessor,
-        Object... args) throws IgniteCheckedException {
-        A.notNull(keys, "keys", entryProcessor, "entryProcessor");
-
-        warnIfUnordered(keys, BulkOperation.INVOKE);
-
-        final boolean statsEnabled = ctx.statisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        Map<? extends K, EntryProcessor> invokeMap = F.viewAsMap(keys, new C1<K, EntryProcessor>() {
-            @Override public EntryProcessor apply(K k) {
-                return entryProcessor;
-            }
-        });
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
-
-        Map<K, EntryProcessorResult<T>> entryProcessorRes = (Map<K, EntryProcessorResult<T>>)updateAllInternal(
-                TRANSFORM,
-                invokeMap.keySet(),
-                invokeMap.values(),
-                args,
-                expiryPerCall(),
-                false,
-                false,
-                null,
-                ctx.writeThrough(),
-                ctx.readThrough(),
-                keepBinary);
-
-        if (statsEnabled)
-            metrics0().addInvokeTimeNanos(System.nanoTime() - start);
-
-        return entryProcessorRes;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
-        EntryProcessor<K, V, T> entryProcessor,
-        Object... args) throws EntryProcessorException {
-        A.notNull(key, "key", entryProcessor, "entryProcessor");
-
-        final boolean statsEnabled = ctx.statisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        Map<? extends K, EntryProcessor> invokeMap =
-            Collections.singletonMap(key, (EntryProcessor)entryProcessor);
-
-        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAllAsync0(null,
-            invokeMap,
-            args,
-            false,
-            false,
-            null);
-
-        return fut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
-            @Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut)
-                throws IgniteCheckedException {
-                Map<K, EntryProcessorResult<T>> resMap = fut.get();
-
-                if (statsEnabled)
-                    metrics0().addInvokeTimeNanos(System.nanoTime() - start);
-
-                if (resMap != null) {
-                    assert resMap.isEmpty() || resMap.size() == 1 : resMap.size();
-
-                    return resMap.isEmpty() ? new CacheInvokeResult<>() : resMap.values().iterator().next();
-                }
-
-                return new CacheInvokeResult<>();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
-        Set<? extends K> keys,
-        final EntryProcessor<K, V, T> entryProcessor,
-        Object... args) {
-        A.notNull(keys, "keys", entryProcessor, "entryProcessor");
-
-        warnIfUnordered(keys, BulkOperation.INVOKE);
-
-        final boolean statsEnabled = ctx.statisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        Map<? extends K, EntryProcessor> invokeMap = F.viewAsMap(keys, new C1<K, EntryProcessor>() {
-            @Override public EntryProcessor apply(K k) {
-                return entryProcessor;
-            }
-        });
-
-        IgniteInternalFuture fut = updateAllAsync0(null,
-            invokeMap,
-            args,
-            true,
-            false,
-            null);
-
-        if (statsEnabled)
-            fut.listen(new InvokeAllTimeStatClosure(metrics0(), start));
-
-        return fut;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(
-        Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
-        Object... args) throws IgniteCheckedException {
-        A.notNull(map, "map");
-
-        warnIfUnordered(map, BulkOperation.INVOKE);
-
-        final boolean statsEnabled = ctx.statisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        Map<K, EntryProcessorResult<T>> entryProcessorResult = (Map<K, EntryProcessorResult<T>>)updateAllInternal(
-                TRANSFORM,
-                map.keySet(),
-                map.values(),
-                args,
-                expiryPerCall(),
-                false,
-                false,
-                null,
-                ctx.writeThrough(),
-                ctx.readThrough(),
-                opCtx != null && opCtx.isKeepBinary());
-
-        if (statsEnabled)
-            metrics0().addInvokeTimeNanos(System.nanoTime() - start);
-
-        return entryProcessorResult;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
-        Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
-        Object... args) {
-        A.notNull(map, "map");
-
-        warnIfUnordered(map, BulkOperation.INVOKE);
-
-        final boolean statsEnabled = ctx.statisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        IgniteInternalFuture fut = updateAllAsync0(null,
-            map,
-            args,
-            true,
-            false,
-            null);
-
-        if (statsEnabled)
-            fut.listen(new InvokeAllTimeStatClosure(metrics0(), start));
-
-        return fut;
-    }
-
-    /**
-     * Entry point for public API update methods.
-     *
-     * @param map Put map. Either {@code map} or {@code invokeMap} should be passed.
-     * @param invokeMap Transform map. Either {@code map} or {@code invokeMap} should be passed.
-     * @param invokeArgs Optional arguments for EntryProcessor.
-     * @param retval Return value required flag.
-     * @param rawRetval Return {@code GridCacheReturn} instance.
-     * @param filter Cache entry filter for atomic updates.
-     * @return Completion future.
-     */
-    private IgniteInternalFuture updateAllAsync0(
-        @Nullable final Map<? extends K, ? extends V> map,
-        @Nullable final Map<? extends K, ? extends EntryProcessor> invokeMap,
-        @Nullable final Object[] invokeArgs,
-        final boolean retval,
-        final boolean rawRetval,
-        @Nullable final CacheEntryPredicate filter
-    ) {
-        final GridCacheOperation op = invokeMap != null ? TRANSFORM : UPDATE;
-
-        final Collection<? extends K> keys =
-            map != null ? map.keySet() : invokeMap != null ? invokeMap.keySet() : null;
-
-        final Collection<?> vals = map != null ? map.values() : invokeMap != null ? invokeMap.values() : null;
-
-        final boolean writeThrough = ctx.writeThrough();
-
-        final boolean readThrough = ctx.readThrough();
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        final ExpiryPolicy expiry = expiryPerCall();
-
-        final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
-
-        return asyncOp(new GridPlainCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return updateAllInternal(op,
-                    keys,
-                    vals,
-                    invokeArgs,
-                    expiry,
-                    retval,
-                    rawRetval,
-                    filter,
-                    writeThrough,
-                    readThrough,
-                    keepBinary);
-            }
-        });
-    }
-
-    /**
-     * Entry point for public API remove methods.
-     *
-     * @param keys Keys to remove.
-     * @param retval Return value required flag.
-     * @param rawRetval Return {@code GridCacheReturn} instance.
-     * @param filter Cache entry filter.
-     * @return Completion future.
-     */
-    private IgniteInternalFuture removeAllAsync0(
-        @Nullable final Collection<? extends K> keys,
-        final boolean retval,
-        final boolean rawRetval,
-        @Nullable final CacheEntryPredicate filter
-    ) {
-        final boolean writeThrough = ctx.writeThrough();
-
-        final boolean readThrough = ctx.readThrough();
-
-        final ExpiryPolicy expiryPlc = expiryPerCall();
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
-
-        return asyncOp(new GridPlainCallable<Object>() {
-            @Override public Object call() throws Exception {
-                return updateAllInternal(DELETE,
-                    keys,
-                    null,
-                    null,
-                    expiryPlc,
-                    retval,
-                    rawRetval,
-                    filter,
-                    writeThrough,
-                    readThrough,
-                    keepBinary);
-            }
-        });
-    }
-
-    /**
-     * Entry point for all public update methods (put, remove, invoke).
-     *
-     * @param op Operation.
-     * @param keys Keys.
-     * @param vals Values.
-     * @param invokeArgs Optional arguments for EntryProcessor.
-     * @param expiryPlc Expiry policy.
-     * @param retval Return value required flag.
-     * @param rawRetval Return {@code GridCacheReturn} instance.
-     * @param filter Cache entry filter.
-     * @param writeThrough Write through.
-     * @param readThrough Read through.
-     * @return Update result.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("unchecked")
-    private Object updateAllInternal(GridCacheOperation op,
-        Collection<? extends K> keys,
-        @Nullable Iterable<?> vals,
-        @Nullable Object[] invokeArgs,
-        @Nullable ExpiryPolicy expiryPlc,
-        boolean retval,
-        boolean rawRetval,
-        CacheEntryPredicate filter,
-        boolean writeThrough,
-        boolean readThrough,
-        boolean keepBinary
-    ) throws IgniteCheckedException {
-        if (op == DELETE)
-            ctx.checkSecurity(SecurityPermission.CACHE_REMOVE);
-        else
-            ctx.checkSecurity(SecurityPermission.CACHE_PUT);
-
-        String taskName = ctx.kernalContext().job().currentTaskName();
-
-        GridCacheVersion ver = nextVersion();
-
-        CacheEntryPredicate[] filters = CU.filterArray(filter);
-
-        IgniteBiTuple<Boolean, ?> res = null;
-
-        CachePartialUpdateCheckedException err = null;
-
-        ctx.shared().database().checkpointReadLock();
-
-        try {
-            ctx.shared().database().ensureFreeSpace(ctx.dataRegion());
-
-            if (writeThrough && keys.size() > 1) {
-                return updateWithBatch(op,
-                    keys,
-                    vals,
-                    invokeArgs,
-                    expiryPlc,
-                    ver,
-                    filters,
-                    keepBinary,
-                    taskName);
-            }
-
-            Iterator<?> valsIter = vals != null ? vals.iterator() : null;
-
-            boolean intercept = ctx.config().getInterceptor() != null;
-
-            for (K key : keys) {
-                if (key == null)
-                    throw new NullPointerException("Null key.");
-
-                Object val = valsIter != null ? valsIter.next() : null;
-
-                if (val == null && op != DELETE)
-                    throw new NullPointerException("Null value.");
-
-                KeyCacheObject cacheKey = ctx.toCacheKeyObject(key);
-
-                if (op == UPDATE) {
-                    val = ctx.toCacheObject(val);
-
-                    ctx.validateKeyAndValue(cacheKey, (CacheObject)val);
-                }
-                else if (op == TRANSFORM)
-                    ctx.kernalContext().resource().inject(val, GridResourceIoc.AnnotationSet.ENTRY_PROCESSOR, ctx.name());
-
-                while (true) {
-                    GridCacheEntryEx entry = null;
-
-                    try {
-                        entry = entryEx(cacheKey);
-
-                        GridTuple3<Boolean, Object, EntryProcessorResult<Object>> t = entry.innerUpdateLocal(
-                            ver,
-                            val == null ? DELETE : op,
-                            val,
-                            invokeArgs,
-                            writeThrough,
-                            readThrough,
-                            retval,
-                            keepBinary,
-                            expiryPlc,
-                            true,
-                            true,
-                            filters,
-                            intercept,
-                            taskName,
-                            false);
-
-                        if (op == TRANSFORM) {
-                            if (t.get3() != null) {
-                                Map<K, EntryProcessorResult> computedMap;
-
-                                if (res == null) {
-                                    computedMap = U.newHashMap(keys.size());
-
-                                    res = new IgniteBiTuple<>(true, computedMap);
-                                }
-                                else
-                                    computedMap = (Map<K, EntryProcessorResult>)res.get2();
-
-                                computedMap.put(key, t.get3());
-                            }
-                        }
-                        else if (res == null)
-                            res = new T2(t.get1(), t.get2());
-
-                        break; // While.
-                    }
-                    catch (GridCacheEntryRemovedException ignored) {
-                        if (log.isDebugEnabled())
-                            log.debug("Got removed entry while updating (will retry): " + key);
-
-                        entry = null;
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (err == null)
-                            err = partialUpdateException();
-
-                        err.add(F.asList(key), e);
-
-                        U.error(log, "Failed to update key : " + key, e);
-
-                        break;
-                    }
-                    finally {
-                        if (entry != null)
-                            entry.touch();
-                    }
-                }
-            }
-        }
-        finally {
-            ctx.shared().database().checkpointReadUnlock();
-        }
-
-        if (err != null)
-            throw err;
-
-        Object ret = res == null ? null : rawRetval ? new GridCacheReturn(
-            ctx,
-            true,
-            keepBinary,
-            U.deploymentClassLoader(ctx.kernalContext(), U.contextDeploymentClassLoaderId(ctx.kernalContext())),
-            res.get2(),
-            res.get1()
-        ) : (retval || op == TRANSFORM) ? res.get2() : res.get1();
-
-        if (op == TRANSFORM && ret == null)
-            ret = Collections.emptyMap();
-
-        return ret;
-    }
-
-    /**
-     * Updates entries using batched write-through.
-     *
-     * @param op Operation.
-     * @param keys Keys.
-     * @param vals Values.
-     * @param invokeArgs Optional arguments for EntryProcessor.
-     * @param expiryPlc Expiry policy.
-     * @param ver Cache version.
-     * @param filter Optional filter.
-     * @param taskName Task name.
-     * @return Results map for invoke operation.
-     * @throws CachePartialUpdateCheckedException If update failed.
-     */
-    @SuppressWarnings({"ForLoopReplaceableByForEach", "unchecked"})
-    private Map<K, EntryProcessorResult> updateWithBatch(
-        GridCacheOperation op,
-        Collection<? extends K> keys,
-        @Nullable Iterable<?> vals,
-        @Nullable Object[] invokeArgs,
-        @Nullable ExpiryPolicy expiryPlc,
-        GridCacheVersion ver,
-        @Nullable CacheEntryPredicate[] filter,
-        boolean keepBinary,
-        String taskName
-    ) throws IgniteCheckedException {
-        List<GridCacheEntryEx> locked = lockEntries(keys);
-
-        try {
-            int size = locked.size();
-
-            Map<KeyCacheObject, CacheObject> putMap = null;
-
-            Collection<KeyCacheObject> rmvKeys = null;
-
-            List<CacheObject> writeVals = null;
-
-            Map<K, EntryProcessorResult> invokeResMap =
-                op == TRANSFORM ? U.<K, EntryProcessorResult>newHashMap(size) : null;
-
-            List<GridCacheEntryEx> filtered = new ArrayList<>(size);
-
-            CachePartialUpdateCheckedException err = null;
-
-            Iterator<?> valsIter = vals != null ? vals.iterator() : null;
-
-            boolean intercept = ctx.config().getInterceptor() != null;
-
-            for (int i = 0; i < size; i++) {
-                GridCacheEntryEx entry = locked.get(i);
-
-                Object val = valsIter != null ? valsIter.next() : null;
-
-                if (val == null && op != DELETE)
-                    throw new NullPointerException("Null value.");
-
-                try {
-                    try {
-                        if (!ctx.isAllLocked(entry, filter)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Entry did not pass the filter (will skip write) [entry=" + entry +
-                                    ", filter=" + Arrays.toString(filter) + ']');
-
-                            continue;
-                        }
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (err == null)
-                            err = partialUpdateException();
-
-                        err.add(F.asList(entry.key()), e);
-
-                        continue;
-                    }
-
-                    if (op == TRANSFORM) {
-                        ctx.kernalContext().resource().inject(val,
-                            GridResourceIoc.AnnotationSet.ENTRY_PROCESSOR,
-                            ctx.name());
-
-                        EntryProcessor<Object, Object, Object> entryProcessor =
-                            (EntryProcessor<Object, Object, Object>)val;
-
-                        CacheObject old = entry.innerGet(
-                            null,
-                            null,
-                            /*read-through*/true,
-                            /*update-metrics*/true,
-                            /*event*/true,
-                            entryProcessor,
-                            taskName,
-                            null,
-                            keepBinary);
-
-                        Object oldVal = null;
-
-                        CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry<>(entry.key(), old,
-                            entry.version(), keepBinary, entry);
-
-                        CacheObject updated;
-                        Object updatedVal = null;
-                        CacheInvokeResult invokeRes = null;
-
-                        boolean validation = false;
-
-                        IgniteThread.onEntryProcessorEntered(false);
-
-                        try {
-                            Object computed = entryProcessor.process(invokeEntry, invokeArgs);
-
-                            updatedVal = ctx.unwrapTemporary(invokeEntry.getValue());
-
-                            updated = ctx.toCacheObject(updatedVal);
-
-                            if (computed != null)
-                                invokeRes = CacheInvokeResult.fromResult(ctx.unwrapTemporary(computed));
-
-                            if (invokeEntry.modified() && updated != null) {
-                                validation = true;
-
-                                ctx.validateKeyAndValue(entry.key(), updated);
-                            }
-                            else if (ctx.statisticsEnabled() && !invokeEntry.modified())
-                                ctx.cache().metrics0().onReadOnlyInvoke(old != null);
-                        }
-                        catch (Exception e) {
-                            invokeRes = CacheInvokeResult.fromError(e);
-
-                            updated = old;
-
-                            if (validation) {
-                                invokeResMap.put((K)entry.key().value(ctx.cacheObjectContext(), false), invokeRes);
-
-                                continue;
-                            }
-                        }
-                        finally {
-                            IgniteThread.onEntryProcessorLeft();
-                        }
-
-                        if (invokeRes != null)
-                            invokeResMap.put((K)entry.key().value(ctx.cacheObjectContext(), false), invokeRes);
-
-                        if (updated == null) {
-                            if (intercept) {
-                                IgniteBiTuple<Boolean, ?> interceptorRes = ctx.config().getInterceptor()
-                                    .onBeforeRemove(new CacheLazyEntry(ctx, entry.key(), invokeEntry.key(),
-                                        old, oldVal, keepBinary));
-
-                                if (ctx.cancelRemove(interceptorRes))
-                                    continue;
-                            }
-
-                            // Update previous batch.
-                            if (putMap != null) {
-                                err = updatePartialBatch(
-                                    filtered,
-                                    ver,
-                                    writeVals,
-                                    putMap,
-                                    null,
-                                    expiryPlc,
-                                    keepBinary,
-                                    err,
-                                    taskName,
-                                    true);
-
-                                putMap = null;
-                                writeVals = null;
-
-                                filtered = new ArrayList<>();
-                            }
-
-                            // Start collecting new batch.
-                            if (rmvKeys == null)
-                                rmvKeys = new ArrayList<>(size);
-
-                            rmvKeys.add(entry.key());
-                        }
-                        else {
-                            if (intercept) {
-                                Object interceptorVal = ctx.config().getInterceptor()
-                                    .onBeforePut(new CacheLazyEntry(ctx, entry.key(), invokeEntry.getKey(),
-                                        old, oldVal, keepBinary), updatedVal);
-
-                                if (interceptorVal == null)
-                                    continue;
-
-                                updated = ctx.toCacheObject(ctx.unwrapTemporary(interceptorVal));
-                            }
-
-                            // Update previous batch.
-                            if (rmvKeys != null) {
-                                err = updatePartialBatch(
-                                    filtered,
-                                    ver,
-                                    null,
-                                    null,
-                                    rmvKeys,
-                                    expiryPlc,
-                                    keepBinary,
-                                    err,
-                                    taskName,
-                                    true);
-
-                                rmvKeys = null;
-
-                                filtered = new ArrayList<>();
-                            }
-
-                            if (putMap == null) {
-                                putMap = new LinkedHashMap<>(size, 1.0f);
-                                writeVals = new ArrayList<>(size);
-                            }
-
-                            putMap.put(entry.key(), updated);
-                            writeVals.add(updated);
-                        }
-                    }
-                    else if (op == UPDATE) {
-                        CacheObject cacheVal = ctx.toCacheObject(val);
-
-                        if (intercept) {
-                            CacheObject old = entry.innerGet(
-                                null,
-                                null,
-                                /*read-through*/ctx.loadPreviousValue(),
-                                /*update-metrics*/true,
-                                /*event*/true,
-                                null,
-                                taskName,
-                                null,
-                                keepBinary);
-
-                            Object interceptorVal = ctx.config().getInterceptor().onBeforePut(new CacheLazyEntry(
-                                ctx, entry.key(), old, keepBinary), val);
-
-                            if (interceptorVal == null)
-                                continue;
-
-                            cacheVal = ctx.toCacheObject(ctx.unwrapTemporary(interceptorVal));
-                        }
-
-                        ctx.validateKeyAndValue(entry.key(), cacheVal);
-
-                        if (putMap == null) {
-                            putMap = new LinkedHashMap<>(size, 1.0f);
-                            writeVals = new ArrayList<>(size);
-                        }
-
-                        putMap.put(entry.key(), cacheVal);
-                        writeVals.add(cacheVal);
-                    }
-                    else {
-                        assert op == DELETE;
-
-                        if (intercept) {
-                            CacheObject old = entry.innerGet(
-                                null,
-                                null,
-                                /*read-through*/ctx.loadPreviousValue(),
-                                /*update-metrics*/true,
-                                /*event*/true,
-                                null,
-                                taskName,
-                                null,
-                                keepBinary);
-
-                            IgniteBiTuple<Boolean, ?> interceptorRes = ctx.config().getInterceptor()
-                                .onBeforeRemove(new CacheLazyEntry(ctx, entry.key(), old, keepBinary));
-
-                            if (ctx.cancelRemove(interceptorRes))
-                                continue;
-                        }
-
-                        if (rmvKeys == null)
-                            rmvKeys = new ArrayList<>(size);
-
-                        rmvKeys.add(entry.key());
-                    }
-
-                    filtered.add(entry);
-                }
-                catch (IgniteCheckedException e) {
-                    if (err == null)
-                        err = partialUpdateException();
-
-                    err.add(F.asList(entry.key()), e);
-                }
-                catch (GridCacheEntryRemovedException ignore) {
-                    assert false : "Entry cannot become obsolete while holding lock.";
-                }
-            }
-
-            // Store final batch.
-            if (putMap != null || rmvKeys != null) {
-                err = updatePartialBatch(
-                    filtered,
-                    ver,
-                    writeVals,
-                    putMap,
-                    rmvKeys,
-                    expiryPlc,
-                    keepBinary,
-                    err,
-                    taskName,
-                    op == TRANSFORM);
-            }
-            else
-                assert filtered.isEmpty();
-
-            if (err != null)
-                throw err;
-
-            return invokeResMap;
-        }
-        finally {
-            unlockEntries(locked);
-        }
-    }
-
-    /**
-     * @param entries Entries to update.
-     * @param ver Cache version.
-     * @param writeVals Cache values.
-     * @param putMap Values to put.
-     * @param rmvKeys Keys to remove.
-     * @param expiryPlc Expiry policy.
-     * @param err Optional partial update exception.
-     * @param taskName Task name.
-     * @param transformed {@code True} if transform operation performed.
-     * @return Partial update exception.
-     */
-    @SuppressWarnings({"unchecked", "ConstantConditions"})
-    @Nullable private CachePartialUpdateCheckedException updatePartialBatch(
-        List<GridCacheEntryEx> entries,
-        final GridCacheVersion ver,
-        @Nullable List<CacheObject> writeVals,
-        @Nullable Map<KeyCacheObject, CacheObject> putMap,
-        @Nullable Collection<KeyCacheObject> rmvKeys,
-        @Nullable ExpiryPolicy expiryPlc,
-        boolean keepBinary,
-        @Nullable CachePartialUpdateCheckedException err,
-        String taskName,
-        boolean transformed) {
-        assert putMap == null ^ rmvKeys == null;
-        GridCacheOperation op;
-
-        CacheStorePartialUpdateException storeErr = null;
-
-        try {
-            if (putMap != null) {
-                try {
-                    Map<? extends KeyCacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>> view = F.viewReadOnly(putMap,
-                        new C1<CacheObject, IgniteBiTuple<? extends CacheObject, GridCacheVersion>>() {
-                            @Override public IgniteBiTuple<? extends CacheObject, GridCacheVersion> apply(CacheObject val) {
-                                return F.t(val, ver);
-                            }
-                        });
-
-                    ctx.store().putAll(null, view);
-                }
-                catch (CacheStorePartialUpdateException e) {
-                    storeErr = e;
-                }
-
-                op = UPDATE;
-            }
-            else {
-                try {
-                    ctx.store().removeAll(null, rmvKeys);
-                }
-                catch (CacheStorePartialUpdateException e) {
-                    storeErr = e;
-                }
-
-                op = DELETE;
-            }
-        }
-        catch (IgniteCheckedException e) {
-            if (err == null)
-                err = partialUpdateException();
-
-            err.add(putMap != null ? putMap.keySet() : rmvKeys, e);
-
-            return err;
-        }
-
-        boolean intercept = ctx.config().getInterceptor() != null;
-
-        for (int i = 0; i < entries.size(); i++) {
-            GridCacheEntryEx entry = entries.get(i);
-
-            assert entry.lockedByCurrentThread();
-
-            if (entry.obsolete() ||
-                (storeErr != null && storeErr.failedKeys().contains(entry.key().value(ctx.cacheObjectContext(), false))))
-                continue;
-
-            try {
-                // We are holding java-level locks on entries at this point.
-                CacheObject writeVal = op == UPDATE ? writeVals.get(i) : null;
-
-                assert writeVal != null || op == DELETE : "null write value found.";
-
-                GridTuple3<Boolean, Object, EntryProcessorResult<Object>> t = entry.innerUpdateLocal(
-                    ver,
-                    op,
-                    writeVal,
-                    null,
-                    false,
-                    false,
-                    false,
-                    keepBinary,
-                    expiryPlc,
-                    true,
-                    true,
-                    null,
-                    false,
-                    taskName,
-                    transformed);
-
-                if (intercept) {
-                    if (op == UPDATE)
-                        ctx.config().getInterceptor().onAfterPut(new CacheLazyEntry(ctx, entry.key(), writeVal, keepBinary));
-                    else
-                        ctx.config().getInterceptor().onAfterRemove(new CacheLazyEntry(ctx, entry.key(), t.get2(), keepBinary));
-                }
-            }
-            catch (GridCacheEntryRemovedException ignore) {
-                assert false : "Entry cannot become obsolete while holding lock.";
-            }
-            catch (IgniteCheckedException e) {
-                if (err == null)
-                    err = partialUpdateException();
-
-                err.add(Collections.singleton(entry.key()), e);
-            }
-        }
-
-        return err;
-    }
-
-    /**
-     * Acquires java-level locks on cache entries.
-     *
-     * @param keys Keys to lock.
-     * @return Collection of locked entries.
-     */
-    private List<GridCacheEntryEx> lockEntries(Collection<? extends K> keys) {
-        GridCacheEntryEx[] locked = new GridCacheEntryEx[keys.size()];
-
-        boolean nullKeys = false;
-
-        while (true) {
-            int i = 0;
-
-            for (K key : keys) {
-                if (key == null) {
-                    nullKeys = true;
-
-                    break;
-                }
-
-                GridCacheEntryEx entry = entryEx(ctx.toCacheKeyObject(key));
-
-                locked[i++] = entry;
-            }
-
-            if (nullKeys)
-                break;
-
-            if (lockedEntriesInfo.tryLockEntries(locked))
-                return Arrays.asList(locked);
-        }
-
-        assert nullKeys;
-
-        AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
-
-        for (GridCacheEntryEx entry : locked) {
-            if (entry != null)
-                entry.touch();
-        }
-
-        throw new NullPointerException("Null key.");
-    }
-
-    /**
-     * Releases java-level locks on cache entries.
-     *
-     * @param locked Locked entries.
-     */
-    private void unlockEntries(Iterable<GridCacheEntryEx> locked) {
-        for (GridCacheEntryEx entry : locked)
-            entry.unlockEntry();
-
-        AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
-
-        for (GridCacheEntryEx entry : locked)
-            entry.touch();
-    }
-
-    /**
-     * @return New partial update exception.
-     */
-    private static CachePartialUpdateCheckedException partialUpdateException() {
-        return new CachePartialUpdateCheckedException("Failed to update keys (retry update if possible).");
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> txLockAsync(Collection<KeyCacheObject> keys,
-        long timeout,
-        IgniteTxLocalEx tx,
-        boolean isRead,
-        boolean retval,
-        TransactionIsolation isolation,
-        boolean invalidate,
-        long createTtl,
-        long accessTtl) {
-        return new GridFinishedFuture<>(new UnsupportedOperationException("Locks are not supported for " +
-            "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)"));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
-        long timeout) {
-        return new GridFinishedFuture<>(new UnsupportedOperationException("Locks are not supported for " +
-            "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)"));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void unlockAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
-        throw new UnsupportedOperationException("Locks are not supported for " +
-            "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)");
-    }
-
-    /**
-     * @return Expiry policy.
-     */
-    @Nullable private ExpiryPolicy expiryPerCall() {
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        ExpiryPolicy expiry = opCtx != null ? opCtx.expiry() : null;
-
-        if (expiry == null)
-            expiry = ctx.expiry();
-
-        return expiry;
-    }
-
-    /**
-     * @param op Operation closure.
-     * @return Future.
-     */
-    @SuppressWarnings("unchecked")
-    private IgniteInternalFuture asyncOp(final Callable<?> op) {
-        IgniteInternalFuture fail = asyncOpAcquire(/*retry*/false);
-
-        if (fail != null)
-            return fail;
-
-        IgniteInternalFuture f = ctx.closures().callLocalSafe(op);
-
-        f.listen(new CI1<IgniteInternalFuture<?>>() {
-            @Override public void apply(IgniteInternalFuture<?> f) {
-                asyncOpRelease(false);
-            }
-        });
-
-        return f;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeferredDelete(GridCacheEntryEx entry, GridCacheVersion ver) {
-        assert false : "Should not be called";
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 28d58ab5dde..2ca91ab167b 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -1427,9 +1427,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 cctx.kernalContext().pools().getSystemExecutorService(),
                 cctx.cache().cacheGroups(),
                 cacheGroup -> {
-                    if (cacheGroup.isLocal())
-                        return null;
-
                     cctx.database().checkpointReadLock();
 
                     try {
@@ -1789,9 +1786,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         Map<Integer, Set<Integer>> res = new HashMap<>();
 
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal())
-                continue;
-
             for (GridDhtLocalPartition locPart : grp.topology().currentLocalPartitions()) {
                 if (locPart.state() == OWNING && (preferWalRebalance() ||
                     locPart.fullSize() > historicalRebalanceThreshold.getOrDefault(walRebalanceThresholdLegacy)))
@@ -2726,7 +2720,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                         CacheGroupContext ctx = cctx.cache().cacheGroup(rbRec.groupId());
 
-                        if (ctx != null && !ctx.isLocal()) {
+                        if (ctx != null) {
                             GridDhtLocalPartition part = ctx.topology().forceCreatePartition(rbRec.partitionId());
 
                             ctx.offheap().dataStore(part).updateInitialCounter(rbRec.start(), rbRec.range());
@@ -2961,7 +2955,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         if (partId == -1)
             partId = cacheCtx.affinity().partition(dataEntry.key());
 
-        GridDhtLocalPartition locPart = cacheCtx.isLocal() ? null : cacheCtx.topology().forceCreatePartition(partId);
+        GridDhtLocalPartition locPart = cacheCtx.topology().forceCreatePartition(partId);
 
         switch (dataEntry.op()) {
             case CREATE:
@@ -3347,7 +3341,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private static void dumpPartitionsInfo(GridCacheSharedContext cctx,
         IgniteLogger log) throws IgniteCheckedException {
         for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
-            if (grp.isLocal() || !grp.persistenceEnabled())
+            if (!grp.persistenceEnabled())
                 continue;
 
             dumpPartitionsInfo(grp, log);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 6a59a14088a..4668f482620 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -126,7 +126,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.failure.FailureType.CRITICAL_ERROR;
 import static org.apache.ignite.internal.processors.cache.GridCacheTtlManager.DFLT_UNWIND_THROTTLING_TIMEOUT;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.EVICTED;
@@ -399,7 +398,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
     ) throws IgniteCheckedException {
         RowStore rowStore0 = store.rowStore();
 
-        if (rowStore0 != null && (partitionStatesRestored || grp.isLocal())) {
+        if (rowStore0 != null && partitionStatesRestored) {
             ((CacheFreeList)rowStore0.freeList()).saveMetadata(grp.statisticsHolderData());
 
             PartitionMetaStorage<SimpleDataRow> partStore = store.partStorage();
@@ -421,21 +420,19 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                 // localPartition will not acquire writeLock here because create=false.
                 GridDhtLocalPartition part = null;
 
-                if (!grp.isLocal()) {
-                    if (beforeDestroy)
-                        state = GridDhtPartitionState.EVICTED;
-                    else {
-                        part = getPartition(store);
-
-                        if (part != null && part.state() != GridDhtPartitionState.EVICTED)
-                            state = part.state();
-                    }
+                if (beforeDestroy)
+                    state = GridDhtPartitionState.EVICTED;
+                else {
+                    part = getPartition(store);
 
-                    // Do not save meta for evicted partitions on next checkpoints.
-                    if (state == null)
-                        return;
+                    if (part != null && part.state() != GridDhtPartitionState.EVICTED)
+                        state = part.state();
                 }
 
+                // Do not save meta for evicted partitions on next checkpoints.
+                if (state == null)
+                    return;
+
                 int grpId = grp.groupId();
                 long partMetaId = pageMem.partitionMetaPageId(grpId, store.partId());
 
@@ -520,10 +517,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                             }
                         }
 
-                        if (state != null)
-                            changed |= io.setPartitionState(partMetaPageAddr, (byte)state.ordinal());
-                        else
-                            assert grp.isLocal() : grp.cacheOrGroupName();
+                        changed |= io.setPartitionState(partMetaPageAddr, (byte)state.ordinal());
 
                         long cntrsPageId;
 
@@ -624,7 +618,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
     /** {@inheritDoc} */
     @Override public long restoreStateOfPartition(int p, @Nullable Integer recoveryState) throws IgniteCheckedException {
-        if (grp.isLocal() || !grp.affinityNode() || !grp.dataRegion().config().isPersistenceEnabled()
+        if (!grp.affinityNode() || !grp.dataRegion().config().isPersistenceEnabled()
             || partitionStatesRestored)
             return 0;
 
@@ -744,7 +738,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
     /** {@inheritDoc} */
     @Override public void restorePartitionStates() throws IgniteCheckedException {
-        if (grp.isLocal() || !grp.affinityNode() || !grp.dataRegion().config().isPersistenceEnabled()
+        if (!grp.affinityNode() || !grp.dataRegion().config().isPersistenceEnabled()
             || partitionStatesRestored)
             return;
 
@@ -1284,12 +1278,6 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
     /** {@inheritDoc} */
     @Override public void preloadPartition(int partId) throws IgniteCheckedException {
-        if (grp.isLocal()) {
-            dataStore(null).preload();
-
-            return;
-        }
-
         GridDhtLocalPartition locPart = grp.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false);
 
         assert locPart != null && locPart.reservations() > 0;
@@ -3090,13 +3078,11 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         ) throws IgniteCheckedException {
             GridDhtLocalPartition part = null;
 
-            if (!grp.isLocal()) {
-                part = cctx.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false);
+            part = cctx.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false);
 
-                // Skip non-owned partitions.
-                if (part == null || part.state() != OWNING || !cctx.topology().initialized())
-                    return 0;
-            }
+            // Skip non-owned partitions.
+            if (part == null || part.state() != OWNING || !cctx.topology().initialized())
+                return 0;
 
             cctx.shared().database().checkpointReadLock();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java
index 215a1acd506..19a8c72a2e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/checkpoint/CheckpointWorkflow.java
@@ -379,7 +379,7 @@ public class CheckpointWorkflow {
         GridCompoundFuture grpHandleFut = checkpointCollectPagesInfoPool == null ? null : new GridCompoundFuture();
 
         for (CacheGroupContext grp : cacheGroupsContexts.get()) {
-            if (grp.isLocal() || !grp.walEnabled())
+            if (!grp.walEnabled())
                 continue;
 
             Runnable r = () -> {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java
index f743049b34a..1d5b920d0b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/migration/UpgradePendingTreeToPerPartitionTask.java
@@ -305,7 +305,7 @@ public class UpgradePendingTreeToPerPartitionTask implements IgniteCallable<Bool
                 assert PageIO.getVersion(pageAddr) != 0;
 
                 IgniteCacheOffheapManager.CacheDataStore store =
-                    grp.offheap().dataStore(grp.isLocal() ? null : grp.topology().localPartition(partition));
+                    grp.offheap().dataStore(grp.topology().localPartition(partition));
 
                 if (store == null) {
                     log.warning("Failed to move old-version pending entry " +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index f0f7307843d..0c6dbe3e1b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -54,8 +54,6 @@ import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteReducer;
 import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
@@ -106,8 +104,6 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
-        assert cctx.config().getCacheMode() != LOCAL;
-
         cctx.io().addCacheHandler(cctx.cacheId(), GridCacheQueryRequest.class, new CI2<UUID, GridCacheQueryRequest>() {
             @Override public void apply(UUID nodeId, GridCacheQueryRequest req) {
                 processQueryRequest(nodeId, req);
@@ -504,8 +500,6 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
     /** */
     private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
         if (log.isDebugEnabled())
             log.debug("Executing distributed query: " + qry);
 
@@ -551,7 +545,6 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
     @SuppressWarnings({"unchecked"})
     @Override public GridCloseableIterator scanQueryDistributed(final GridCacheQueryAdapter qry,
         Collection<ClusterNode> nodes) throws IgniteCheckedException {
-        assert !cctx.isLocal() : cctx.name();
         assert qry.type() == GridCacheQueryType.SCAN : qry;
         assert qry.mvccSnapshot() != null || !cctx.mvccEnabled();
 
@@ -679,8 +672,6 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
     /** {@inheritDoc} */
     @Override public CacheQueryFuture<?> queryFieldsLocal(GridCacheQueryBean qry) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
         if (log.isDebugEnabled())
             log.debug("Executing query on local node: " + qry);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
deleted file mode 100644
index 1e1f2f03d06..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryManager.java
+++ /dev/null
@@ -1,131 +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.internal.processors.cache.query;
-
-import java.util.Collection;
-import java.util.List;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta;
-import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
-import org.apache.ignite.internal.util.lang.GridCloseableIterator;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-
-/**
- * Local query manager (for cache in LOCAL cache mode).
- */
-public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V> {
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected boolean onPageReady(
-        boolean loc,
-        GridCacheQueryInfo qryInfo,
-        IndexQueryResultMeta metadata,
-        Collection<?> data,
-        boolean finished, Throwable e) {
-        GridCacheQueryFutureAdapter fut = qryInfo.localQueryFuture();
-
-        assert fut != null;
-
-        if (e != null)
-            fut.onPage(null, null, null, e, true);
-        else
-            fut.onPage(null, metadata, data, null, finished);
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean onFieldsPageReady(boolean loc,
-        GridCacheQueryInfo qryInfo,
-        @Nullable List<GridQueryFieldMetadata> metaData,
-        @Nullable Collection<?> entities,
-        @Nullable Collection<?> data,
-        boolean finished,
-        @Nullable Throwable e) {
-        assert qryInfo != null;
-
-        GridCacheLocalFieldsQueryFuture fut = (GridCacheLocalFieldsQueryFuture)qryInfo.localQueryFuture();
-
-        assert fut != null;
-
-        if (e != null)
-            fut.onPage(null, null, null, e, true);
-        else
-            fut.onFieldsPage(null, metaData, data, null, finished);
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start0() throws IgniteCheckedException {
-        super.start0();
-
-        assert cctx.config().getCacheMode() == LOCAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() == LOCAL;
-
-        throw new IgniteException("Distributed queries are not available for local cache " +
-            "(use 'CacheQuery.execute(grid.forLocal())' instead) [cacheName=" + cctx.name() + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridCloseableIterator scanQueryDistributed(GridCacheQueryAdapter qry,
-        Collection<ClusterNode> nodes) throws IgniteCheckedException {
-        assert cctx.isLocal() : cctx.name();
-
-        throw new IgniteException("Distributed scan query are not available for local cache " +
-            "(use 'CacheQuery.executeScanQuery(grid.forLocal())' instead) [cacheName=" + cctx.name() + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsLocal(GridCacheQueryBean qry) {
-        assert cctx.config().getCacheMode() == LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing query on local node: " + qry);
-
-        GridCacheLocalFieldsQueryFuture fut = new GridCacheLocalFieldsQueryFuture(cctx, qry);
-
-        try {
-            qry.query().validate();
-
-            fut.execute();
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() == LOCAL;
-
-        throw new IgniteException("Distributed queries are not available for local cache " +
-            "(use 'CacheQuery.execute(grid.forLocal())' instead) [cacheName=" + cctx.name() + ']');
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 6071b8e33a1..72c663327fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -64,15 +64,12 @@ import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.INDEX;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SCAN;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SET;
@@ -590,17 +587,15 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     @Override public GridCloseableIterator executeScanQuery() throws IgniteCheckedException {
         assert type == SCAN : "Wrong processing of query: " + type;
 
-        if (!cctx.isLocal()) {
-            GridDhtCacheAdapter<?, ?> cacheAdapter = cctx.isNear() ? cctx.near().dht() : cctx.dht();
+        GridDhtCacheAdapter<?, ?> cacheAdapter = cctx.isNear() ? cctx.near().dht() : cctx.dht();
 
-            Set<Integer> lostParts = cacheAdapter.topology().lostPartitions();
+        Set<Integer> lostParts = cacheAdapter.topology().lostPartitions();
 
-            if (!lostParts.isEmpty()) {
-                if (part == null || lostParts.contains(part)) {
-                    throw new CacheException(new CacheInvalidStateException("Failed to execute query because cache partition " +
-                        "has been lostParts [cacheName=" + cctx.name() +
-                        ", part=" + (part == null ? lostParts.iterator().next() : part) + ']'));
-                }
+        if (!lostParts.isEmpty()) {
+            if (part == null || lostParts.contains(part)) {
+                throw new CacheException(new CacheInvalidStateException("Failed to execute query because cache partition " +
+                    "has been lostParts [cacheName=" + cctx.name() +
+                    ", part=" + (part == null ? lostParts.iterator().next() : part) + ']'));
             }
         }
 
@@ -669,17 +664,6 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         Integer part = partition();
 
         switch (cacheMode) {
-            case LOCAL:
-                if (prj != null)
-                    U.warn(log, "Ignoring query projection because it's executed over LOCAL cache " +
-                        "(only local node will be queried): " + this);
-
-                if (type == SCAN && cctx.config().getCacheMode() == LOCAL &&
-                    part != null && part >= cctx.affinity().partitions())
-                    throw new IgniteCheckedException("Invalid partition number: " + part);
-
-                return Collections.singletonList(cctx.localNode());
-
             case REPLICATED:
                 if (prj != null || part != null)
                     return nodes(cctx, prj, part);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index a4bf962ec9a..31f02b8cc61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -135,9 +135,7 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilterImpl;
 import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
@@ -817,7 +815,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             else
                 injectResources(keyValFilter);
 
-            Integer part = cctx.isLocal() ? null : qry.partition();
+            Integer part = qry.partition();
 
             if (part != null && (part < 0 || part >= cctx.affinity().partitions()))
                 return new GridEmptyCloseableIterator() {
@@ -861,15 +859,13 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             else {
                 locPart = null;
 
-                if (!cctx.isLocal()) {
-                    final GridDhtCacheAdapter dht = cctx.isNear() ? cctx.near().dht() : cctx.dht();
+                final GridDhtCacheAdapter dht = cctx.isNear() ? cctx.near().dht() : cctx.dht();
 
-                    Set<Integer> lostParts = dht.topology().lostPartitions();
+                Set<Integer> lostParts = dht.topology().lostPartitions();
 
-                    if (!lostParts.isEmpty()) {
-                        throw new CacheInvalidStateException("Failed to execute scan query because cache partition " +
-                            "has been lost [cacheName=" + cctx.name() + ", part=" + lostParts.iterator().next() + "]");
-                    }
+                if (!lostParts.isEmpty()) {
+                    throw new CacheInvalidStateException("Failed to execute scan query because cache partition " +
+                        "has been lost [cacheName=" + cctx.name() + ", part=" + lostParts.iterator().next() + "]");
                 }
 
                 it = cctx.offheap().cacheIterator(cctx.cacheId(), true, backups, topVer,
@@ -3124,7 +3120,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             // keep binary for remote scans if possible
             keepBinary = (!locNode && scanFilter == null && transformer == null && !readEvt) || qry.keepBinary();
             transform = transformer;
-            dht = cctx.isLocal() ? null : (cctx.isNear() ? cctx.near().dht() : cctx.dht());
+            dht = cctx.isNear() ? cctx.near().dht() : cctx.dht();
             cache = dht != null ? dht : cctx.cache();
             objCtx = cctx.cacheObjectContext();
             cacheName = cctx.name();
@@ -3228,8 +3224,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 // Filter backups for SCAN queries, if it isn't partition scan.
                 // Other types are filtered in indexing manager.
-                if (!cctx.isReplicated() && /*qry.partition()*/this.locPart == null &&
-                    cctx.config().getCacheMode() != LOCAL && !incBackups &&
+                if (!cctx.isReplicated() && /*qry.partition()*/this.locPart == null && !incBackups &&
                     !cctx.affinity().primaryByKey(cctx.localNode(), key, topVer)) {
                     if (log.isDebugEnabled())
                         log.debug("Ignoring backup element [row=" + row +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index dc71affd905..69c0a59c169 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -423,21 +423,21 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
             @Override public void onBeforeRegister() {
                 GridCacheContext<K, V> cctx = cacheContext(ctx);
 
-                if (cctx != null && !cctx.isLocal())
+                if (cctx != null)
                     cctx.topology().readLock();
             }
 
             @Override public void onAfterRegister() {
                 GridCacheContext<K, V> cctx = cacheContext(ctx);
 
-                if (cctx != null && !cctx.isLocal())
+                if (cctx != null)
                     cctx.topology().readUnlock();
             }
 
             @Override public void onRegister() {
                 GridCacheContext<K, V> cctx = cacheContext(ctx);
 
-                if (cctx != null && !cctx.isLocal())
+                if (cctx != null)
                     locInitUpdCntrs = toCountersMap(cctx.topology().localUpdateCounters(false));
             }
 
@@ -681,7 +681,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
                 assert !skipEvt || evt == null;
                 assert skipEvt || part == -1 && cntr == -1; // part == -1 && cntr == -1 means skip counter.
 
-                if (!cctx.mvccEnabled() || cctx.isLocal())
+                if (!cctx.mvccEnabled())
                     return true;
 
                 assert locInitUpdCntrs != null;
@@ -857,14 +857,12 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
     void waitTopologyFuture(GridKernalContext ctx) throws IgniteCheckedException {
         GridCacheContext<K, V> cctx = cacheContext(ctx);
 
-        if (!cctx.isLocal()) {
-            AffinityTopologyVersion topVer = initTopVer;
+        AffinityTopologyVersion topVer = initTopVer;
 
-            cacheContext(ctx).shared().exchange().affinityReadyFuture(topVer).get();
+        cacheContext(ctx).shared().exchange().affinityReadyFuture(topVer).get();
 
-            for (int partId = 0; partId < cacheContext(ctx).affinity().partitions(); partId++)
-                getOrCreatePartitionRecovery(ctx, partId, topVer);
-        }
+        for (int partId = 0; partId < cacheContext(ctx).affinity().partitions(); partId++)
+            getOrCreatePartitionRecovery(ctx, partId, topVer);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index f680e375052..6d5253c2e90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -207,8 +207,7 @@ public class CacheContinuousQueryManager<K, V> extends GridCacheManagerAdapter<K
      * @return {@code True} if should notify continuous query manager.
      */
     public boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) {
-        return cctx.isLocal() ||
-            cctx.isReplicated() ||
+        return cctx.isReplicated() ||
             (!cctx.isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local()));
     }
 
@@ -738,17 +737,14 @@ public class CacheContinuousQueryManager<K, V> extends GridCacheManagerAdapter<K
 
         final CacheContinuousQueryHandler hnd = clsr.apply();
 
-        boolean locOnly = cctx.isLocal() || loc;
-
         hnd.taskNameHash(taskNameHash);
         hnd.skipPrimaryCheck(skipPrimaryCheck);
         hnd.notifyExisting(notifyExisting);
         hnd.internal(internal);
         hnd.keepBinary(keepBinary);
-        hnd.localOnly(locOnly);
+        hnd.localOnly(loc);
 
-        IgnitePredicate<ClusterNode> pred = (loc || cctx.config().getCacheMode() == CacheMode.LOCAL)
-            ? F.nodeForNodeId(cctx.localNodeId())
+        IgnitePredicate<ClusterNode> pred = loc ? F.nodeForNodeId(cctx.localNodeId())
             : new IsAllPredicate<>(cctx.group().nodeFilter(), new AttributeNodeFilter(ATTR_CLIENT_MODE, false));
 
         assert pred != null : cctx.config();
@@ -758,13 +754,13 @@ public class CacheContinuousQueryManager<K, V> extends GridCacheManagerAdapter<K
         try {
             id = cctx.kernalContext().continuous().startRoutine(
                 hnd,
-                locOnly,
+                loc,
                 bufSize,
                 timeInterval,
                 autoUnsubscribe,
                 pred).get();
 
-            if (hnd.isQuery() && cctx.userCache() && !locOnly && !onStart)
+            if (hnd.isQuery() && cctx.userCache() && !loc && !onStart)
                 hnd.waitTopologyFuture(cctx.kernalContext());
         }
         catch (NodeStoppingException e) {
@@ -1000,7 +996,7 @@ public class CacheContinuousQueryManager<K, V> extends GridCacheManagerAdapter<K
                         lsnr.onRegister();
 
                         if (lsnrCnt.get() == 1) {
-                            if (grp.sharedGroup() && !cctx.isLocal())
+                            if (grp.sharedGroup())
                                 grp.addCacheWithContinuousQuery(cctx);
                         }
                     }
@@ -1039,7 +1035,7 @@ public class CacheContinuousQueryManager<K, V> extends GridCacheManagerAdapter<K
                 if ((lsnr = lsnrs.remove(id)) != null) {
                     int cnt = lsnrCnt.decrementAndGet();
 
-                    if (cctx.group().sharedGroup() && cnt == 0 && !cctx.isLocal())
+                    if (cctx.group().sharedGroup() && cnt == 0)
                         cctx.group().removeCacheWithContinuousQuery(cctx);
                 }
             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
index 97b2b7bc452..e2d04de476a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
@@ -113,7 +113,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
                     boolean start = ignite.configuration().isClientMode();
 
                     IgniteCache<?, ?> cache0 =
-                            ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start, false);
+                            ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start);
 
                     if (cache0 != null)
                         node = mapToNode(subgrid, args, first, cache0.getName());
@@ -257,7 +257,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
                     try {
                         boolean start = ignite.configuration().isClientMode();
 
-                        cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start, false);
+                        cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start);
                     }
                     catch (IgniteCheckedException e) {
                         throw new IgniteException(e);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
index fb6dc558e88..62411380e3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
@@ -156,7 +156,7 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter {
 
     /** {@inheritDoc} */
     @Override public GridDhtTopologyFuture topologyReadLock(GridCacheSharedContext cctx, GridFutureAdapter<?> fut) {
-        if (cacheCtx == null || cacheCtx.isLocal())
+        if (cacheCtx == null)
             return cctx.exchange().lastTopologyFuture();
 
         cacheCtx.topology().readLock();
@@ -175,7 +175,7 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter {
 
     /** {@inheritDoc} */
     @Override public void topologyReadUnlock(GridCacheSharedContext cctx) {
-        if (cacheCtx == null || cacheCtx.isLocal())
+        if (cacheCtx == null)
             return;
 
         cacheCtx.topology().readUnlock();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index dc3da4c8e4a..7b67d58906d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -1201,15 +1201,11 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                     if (retval || invoke) {
                         if (!cacheCtx.isNear()) {
                             if (!hasPrevVal) {
-                                // For non-local cache should read from store after lock on primary.
-                                boolean readThrough = cacheCtx.isLocal() &&
-                                    (invoke || cacheCtx.loadPreviousValue()) &&
-                                    !txEntry.skipStore();
-
+                                // For caches, we should read from store after lock on primary.
                                 v = cached.innerGet(
                                     null,
                                     this,
-                                    readThrough,
+                                    false,
                                     /*metrics*/!invoke,
                                     /*event*/!invoke && !dht(),
                                     null,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
index 00d4817d6e9..c3f39375a1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
@@ -282,12 +282,10 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter {
 
             GridCacheContext<?, ?> cacheCtx = cctx.cacheContext(cacheId);
 
-            if (!cacheCtx.isLocal()) {
-                if (nonLocCtx == null)
-                    nonLocCtx = cacheCtx;
+            if (nonLocCtx == null)
+                nonLocCtx = cacheCtx;
 
-                cacheCtxs.putIfAbsent(cacheCtx.cacheId(), cacheCtx);
-            }
+            cacheCtxs.putIfAbsent(cacheCtx.cacheId(), cacheCtx);
         }
 
         if (nonLocCtx == null)
@@ -321,11 +319,9 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter {
 
                 GridCacheContext<?, ?> cacheCtx = cctx.cacheContext(cacheId);
 
-                if (!cacheCtx.isLocal()) {
-                    nonLocCtx = cacheCtx;
+                nonLocCtx = cacheCtx;
 
-                    break;
-                }
+                break;
             }
 
             if (nonLocCtx != null)
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java
index 01231990050..4b2aeee56d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/CacheInfo.java
@@ -359,7 +359,7 @@ public class CacheInfo extends VisorDataTransferObject {
         out.writeInt(partitions);
         out.writeInt(mapped);
         out.writeObject(topVer);
-        U.writeEnum(out, mode);
+        out.writeByte(CacheMode.toCode(mode));
         out.writeInt(backupsCnt);
         U.writeString(out, affinityClsName);
         out.writeInt(cachesCnt);
@@ -377,7 +377,7 @@ public class CacheInfo extends VisorDataTransferObject {
         partitions = in.readInt();
         mapped = in.readInt();
         topVer = (AffinityTopologyVersion)in.readObject();
-        mode = CacheMode.fromOrdinal(in.readByte());
+        mode = CacheMode.fromCode(in.readByte());
         backupsCnt = in.readInt();
         affinityClsName = U.readString(in);
         cachesCnt = in.readInt();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java
index f14ab77ce70..885dceeeaf6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTask.java
@@ -222,7 +222,7 @@ public class VerifyBackupPartitionsTask extends ComputeTaskAdapter<Set<String>,
                 Collection<CacheGroupContext> groups = ignite.context().cache().cacheGroups();
 
                 for (CacheGroupContext grp : groups) {
-                    if (!grp.systemCache() && !grp.isLocal())
+                    if (!grp.systemCache())
                         grpIds.add(grp.groupId());
                 }
             }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java
index 0376d81e02c..f75a2b6c70f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/VerifyBackupPartitionsTaskV2.java
@@ -62,9 +62,7 @@ import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static java.util.Collections.emptyMap;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
 import static org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.GRID_NOT_IDLE_MSG;
 import static org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.calculatePartitionHash;
@@ -389,7 +387,7 @@ public class VerifyBackupPartitionsTaskV2 extends ComputeTaskAdapter<VisorIdleVe
             for (GridCacheContext cacheCtx : grp.caches()) {
                 DynamicCacheDescriptor desc = ignite.context().cache().cacheDescriptor(cacheCtx.name());
 
-                if (desc.cacheConfiguration().getCacheMode() != LOCAL && isCacheMatchFilter(desc))
+                if (isCacheMatchFilter(desc))
                     return true;
             }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 0fa1f11e409..dd626ccdc48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -1463,7 +1463,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                         if (proc != null) {
                             GridCacheAdapter cache = ctx.cache().internalCache(hnd.cacheName());
 
-                            if (cache != null && !cache.isLocal() && cache.context().userCache())
+                            if (cache != null && cache.context().userCache())
                                 req.addUpdateCounters(ctx.localNodeId(), hnd.updateCounters());
                         }
                     }
@@ -1621,7 +1621,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                             if (proc != null) {
                                 GridCacheAdapter cache = ctx.cache().internalCache(hnd.cacheName());
 
-                                if (cache != null && !cache.isLocal() && cache.context().userCache()) {
+                                if (cache != null && cache.context().userCache()) {
                                     CachePartitionPartialCountersMap cntrsMap =
                                         cache.context().topology().localUpdateCounters(false);
 
@@ -2651,7 +2651,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                         GridCacheContext cctx = interCache != null ? interCache.context() : null;
 
-                        if (cctx != null && cntrsPerNode != null && !cctx.isLocal() && cctx.affinityNode())
+                        if (cctx != null && cntrsPerNode != null && cctx.affinityNode())
                             cntrsPerNode.put(ctx.localNodeId(),
                                 toCountersMap(cctx.topology().localUpdateCounters(false)));
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 3ddab1abb2d..1dca24a1647 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -53,7 +53,6 @@ import org.apache.ignite.IgniteDataStreamerTimeoutException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -121,7 +120,6 @@ import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.stream.StreamReceiver;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_DATASTREAM;
@@ -377,9 +375,6 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         if (cache == null) { // Possible, cache is not configured on node.
             assert ccfg != null;
 
-            if (ccfg.getCacheMode() == CacheMode.LOCAL)
-                throw new CacheException("Impossible to load Local cache configured remotely.");
-
             ctx.grid().getOrCreateCache(ccfg);
         }
 
@@ -845,25 +840,21 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             AffinityTopologyVersion topVer;
 
-            if (!cctx.isLocal()) {
-                GridDhtPartitionsExchangeFuture exchFut = ctx.cache().context().exchange().lastTopologyFuture();
+            GridDhtPartitionsExchangeFuture exchFut = ctx.cache().context().exchange().lastTopologyFuture();
 
-                if (!exchFut.isDone()) {
-                    ExchangeActions acts = exchFut.exchangeActions();
-
-                    if (acts != null && acts.cacheStopped(CU.cacheId(cacheName)))
-                        throw new CacheStoppedException(cacheName);
-                }
+            if (!exchFut.isDone()) {
+                ExchangeActions acts = exchFut.exchangeActions();
 
-                // It is safe to block here even if the cache gate is acquired.
-                topVer = exchFut.get();
+                if (acts != null && acts.cacheStopped(CU.cacheId(cacheName)))
+                    throw new CacheStoppedException(cacheName);
             }
-            else
-                topVer = ctx.cache().context().exchange().readyAffinityVersion();
+
+            // It is safe to block here even if the cache gate is acquired.
+            topVer = exchFut.get();
 
             List<List<ClusterNode>> assignments = cctx.affinity().assignments(topVer);
 
-            if (!allowOverwrite() && !cctx.isLocal()) { // Cases where cctx required.
+            if (!allowOverwrite()) { // Cases where cctx required.
                 gate = cctx.gate();
 
                 gate.enter();
@@ -1113,9 +1104,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         List<ClusterNode> res = null;
 
         if (!allowOverwrite())
-            res = cctx.isLocal() ?
-                aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer) :
-                cctx.topology().nodes(cctx.affinity().partition(key), topVer);
+            res = cctx.topology().nodes(cctx.affinity().partition(key), topVer);
         else {
             ClusterNode node = aff.mapKeyToNode(cacheName, key, topVer);
 
@@ -1806,7 +1795,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             try {
                 GridCacheContext cctx = ctx.cache().internalCache(cacheName).context();
 
-                final boolean lockTop = !cctx.isLocal() && !allowOverwrite();
+                final boolean lockTop = !allowOverwrite();
 
                 GridDhtTopologyFuture topWaitFut = null;
 
@@ -2272,32 +2261,30 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     try {
                         e.getKey().finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
 
-                        if (!cctx.isLocal()) {
-                            int p = cctx.affinity().partition(e.getKey());
+                        int p = cctx.affinity().partition(e.getKey());
 
-                            if (ignoredParts.contains(p))
-                                continue;
+                        if (ignoredParts.contains(p))
+                            continue;
+
+                        if (!reservedParts.contains(p)) {
+                            GridDhtLocalPartition part = cctx.topology().localPartition(p, topVer, true);
 
-                            if (!reservedParts.contains(p)) {
-                                GridDhtLocalPartition part = cctx.topology().localPartition(p, topVer, true);
+                            if (!part.reserve()) {
+                                ignoredParts.add(p);
+
+                                continue;
+                            }
+                            else {
+                                // We must not allow to read from RENTING partitions.
+                                if (part.state() == GridDhtPartitionState.RENTING) {
+                                    part.release();
 
-                                if (!part.reserve()) {
                                     ignoredParts.add(p);
 
                                     continue;
                                 }
-                                else {
-                                    // We must not allow to read from RENTING partitions.
-                                    if (part.state() == GridDhtPartitionState.RENTING) {
-                                        part.release();
 
-                                        ignoredParts.add(p);
-
-                                        continue;
-                                    }
-
-                                    reservedParts.add(p);
-                                }
+                                reservedParts.add(p);
                             }
                         }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index 6417736f3f1..8d95044f186 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -531,7 +531,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     private Collection<ClusterNode> dataNodes(AffinityTopologyVersion topVer) throws IgniteCheckedException {
         assert ctx.isPartitioned() || collocated : "Non-collocated mode is supported only for PARTITIONED caches.";
 
-        if (ctx.isLocal() || (ctx.isReplicated() && ctx.affinityNode()))
+        if (ctx.isReplicated() && ctx.affinityNode())
             return Collections.singleton(ctx.localNode());
 
         Collection<ClusterNode> nodes;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java
index 3450cec3bff..7d3e83ab8a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridSetQueryPredicate.java
@@ -97,7 +97,7 @@ public class GridSetQueryPredicate<K, V> implements IgniteBiPredicate<K, V>, Ext
      * @return {@code True} if need to filter out non-primary keys during processing of set data query.
      */
     private boolean filterKeys() {
-        return !collocated && !(ctx.isLocal() || ctx.isReplicated()) &&
+        return !collocated && !ctx.isReplicated() &&
                 (CU.isNearEnabled(ctx) || ctx.isPartitioned());
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
index 71e62477676..d09f40b00cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
@@ -1839,7 +1839,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
                     if (!cctx.started()) // Cache not started.
                         return reserved;
 
-                    if (cctx.isLocal() || !cctx.rebalanceEnabled())
+                    if (!cctx.rebalanceEnabled())
                         continue;
 
                     boolean checkPartMapping = false;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
index a6e797e3df5..f91c7b73769 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
@@ -49,6 +49,9 @@ public final class ClientStatus {
     /** Cache already exists. */
     public static final int CACHE_EXISTS = 1001;
 
+    /** Invalid cache configuration. */
+    public static final int CACHE_CONFIG_INVALID = 1002;
+
     /** Too many cursors. */
     public static final int TOO_MANY_CURSORS = 1010;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
index c8697f997d8..70c3ca8b551 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
@@ -33,12 +33,13 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.client.ClientException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.client.ClientProtocolContext;
 import org.apache.ignite.internal.processors.platform.client.ClientProtocolVersionFeature;
 import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
-
+import static java.util.Optional.ofNullable;
 import static org.apache.ignite.internal.processors.platform.client.ClientProtocolVersionFeature.QUERY_ENTITY_PRECISION_AND_SCALE;
 
 /**
@@ -155,7 +156,7 @@ public class ClientCacheConfigurationSerializer {
                 CacheConfiguration.DFLT_CACHE_ATOMICITY_MODE);
 
         writer.writeInt(cfg.getBackups());
-        PlatformConfigurationUtils.writeEnumInt(writer, cfg.getCacheMode(), CacheConfiguration.DFLT_CACHE_MODE);
+        writer.writeInt(ofNullable(cfg.getCacheMode()).orElse(CacheConfiguration.DFLT_CACHE_MODE).code());
         writer.writeBoolean(cfg.isCopyOnRead());
         writer.writeString(cfg.getDataRegionName());
         writer.writeBoolean(cfg.isEagerTtl());
@@ -298,7 +299,7 @@ public class ClientCacheConfigurationSerializer {
 
         short propCnt = reader.readShort();
 
-        CacheConfiguration cfg = new CacheConfiguration();
+        CacheConfiguration<?, ?> cfg = new CacheConfiguration<>();
 
         for (int i = 0; i < propCnt; i++) {
             short code = reader.readShort();
@@ -313,7 +314,7 @@ public class ClientCacheConfigurationSerializer {
                     break;
 
                 case CACHE_MODE:
-                    cfg.setCacheMode(CacheMode.fromOrdinal(reader.readInt()));
+                    cfg.setCacheMode(CacheMode.fromCode(reader.readInt()));
                     break;
 
                 case COPY_ON_READ:
@@ -449,6 +450,9 @@ public class ClientCacheConfigurationSerializer {
             }
         }
 
+        if (cfg.getCacheMode() == null)
+            throw new ClientException("Unsupported cache mode");
+
         return cfg;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
index d0ddb4d7b05..e1cd446dde3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import static org.apache.ignite.internal.processors.platform.client.ClientStatus.CACHE_CONFIG_INVALID;
 
 /**
  * Cache create with configuration request.
@@ -49,6 +50,8 @@ public class ClientCacheCreateWithConfigurationRequest extends ClientRequest {
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        checkClientCacheConfiguration(cacheCfg);
+
         try {
             ctx.kernalContext().grid().createCache(cacheCfg);
         }
@@ -58,4 +61,12 @@ public class ClientCacheCreateWithConfigurationRequest extends ClientRequest {
 
         return super.process(ctx);
     }
+
+    /**
+     * @param cfg Cache configuration to validate compatibility.
+     */
+    public static void checkClientCacheConfiguration(CacheConfiguration<?, ?> cfg) {
+        if (cfg.getCacheMode() == null)
+            throw new IgniteClientException(CACHE_CONFIG_INVALID, "The cache mode LOCAL is not supported by the server.");
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
index fc15fc9f40e..df80e211d2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import static org.apache.ignite.internal.processors.platform.client.cache.ClientCacheCreateWithConfigurationRequest.checkClientCacheConfiguration;
 
 /**
  * Cache get or create with configuration request.
@@ -49,6 +50,8 @@ public class ClientCacheGetOrCreateWithConfigurationRequest extends ClientReques
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        checkClientCacheConfiguration(cacheCfg);
+
         try {
             ctx.kernalContext().grid().getOrCreateCache(cacheCfg);
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientAtomicLongCreateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientAtomicLongCreateRequest.java
index 70ef46d2f58..91d0b82a603 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientAtomicLongCreateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientAtomicLongCreateRequest.java
@@ -76,7 +76,7 @@ public class ClientAtomicLongCreateRequest extends ClientRequest {
 
         return new AtomicConfiguration()
                 .setAtomicSequenceReserveSize(reader.readInt())
-                .setCacheMode(CacheMode.fromOrdinal(reader.readByte()))
+                .setCacheMode(CacheMode.fromCode(reader.readByte()))
                 .setBackups(reader.readInt())
                 .setGroupName(reader.readString());
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientIgniteSetGetOrCreateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientIgniteSetGetOrCreateRequest.java
index dcb63c937e4..4efa2b68e51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientIgniteSetGetOrCreateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/datastructures/ClientIgniteSetGetOrCreateRequest.java
@@ -51,7 +51,7 @@ public class ClientIgniteSetGetOrCreateRequest extends ClientRequest {
         collectionConfiguration = create
                 ? new CollectionConfiguration()
                 .setAtomicityMode(CacheAtomicityMode.fromOrdinal(reader.readByte()))
-                .setCacheMode(CacheMode.fromOrdinal(reader.readByte()))
+                .setCacheMode(CacheMode.fromCode(reader.readByte()))
                 .setBackups(reader.readInt())
                 .setGroupName(reader.readString())
                 .setCollocated(reader.readBoolean())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index afc324f9bcf..4a248771a62 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -115,6 +115,7 @@ import org.apache.ignite.ssl.SslContextFactory;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.util.AttributeNodeFilter;
+import static java.util.Optional.ofNullable;
 
 /**
  * Configuration utils.
@@ -178,7 +179,7 @@ public class PlatformConfigurationUtils {
 
         ccfg.setAtomicityMode(CacheAtomicityMode.fromOrdinal(in.readInt()));
         ccfg.setBackups(in.readInt());
-        ccfg.setCacheMode(CacheMode.fromOrdinal(in.readInt()));
+        ccfg.setCacheMode(CacheMode.fromCode(in.readInt()));
         ccfg.setCopyOnRead(in.readBoolean());
         ccfg.setEagerTtl(in.readBoolean());
         ccfg.setInvalidate(in.readBoolean());
@@ -872,7 +873,7 @@ public class PlatformConfigurationUtils {
 
             atomic.setAtomicSequenceReserveSize(in.readInt());
             atomic.setBackups(in.readInt());
-            atomic.setCacheMode(CacheMode.fromOrdinal(in.readInt()));
+            atomic.setCacheMode(CacheMode.fromCode(in.readInt()));
 
             cfg.setAtomicConfiguration(atomic);
         }
@@ -1105,7 +1106,7 @@ public class PlatformConfigurationUtils {
 
         writeEnumInt(writer, ccfg.getAtomicityMode(), CacheConfiguration.DFLT_CACHE_ATOMICITY_MODE);
         writer.writeInt(ccfg.getBackups());
-        writeEnumInt(writer, ccfg.getCacheMode(), CacheConfiguration.DFLT_CACHE_MODE);
+        writer.writeInt(ofNullable(ccfg.getCacheMode()).orElse(CacheConfiguration.DFLT_CACHE_MODE).code());
         writer.writeBoolean(ccfg.isCopyOnRead());
         writer.writeBoolean(ccfg.isEagerTtl());
         writer.writeBoolean(ccfg.isInvalidate());
@@ -1490,7 +1491,7 @@ public class PlatformConfigurationUtils {
 
             w.writeInt(atomic.getAtomicSequenceReserveSize());
             w.writeInt(atomic.getBackups());
-            writeEnumInt(w, atomic.getCacheMode(), AtomicConfiguration.DFLT_CACHE_MODE);
+            w.writeInt(ofNullable(atomic.getCacheMode()).orElse(AtomicConfiguration.DFLT_CACHE_MODE).code());
         }
         else
             w.writeBoolean(false);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e9dc8ef20df..443e12dbb2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -673,15 +673,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             else {
                 CacheConfiguration ccfg = cacheDesc.cacheConfiguration();
 
-                if (ccfg.getCacheMode() == CacheMode.LOCAL) {
-                    // Distributed operation is not allowed on LOCAL caches.
-                    if (log.isDebugEnabled())
-                        log.debug("Received schema propose discovery message, but cache is LOCAL " +
-                            "(will report error) [opId=" + opId + ", msg=" + msg + ']');
-
-                    msg.onError(new SchemaOperationException("Schema changes are not supported for LOCAL cache."));
-                }
-                else if (failOnStaticCacheSchemaChanges(cacheDesc)) {
+                if (failOnStaticCacheSchemaChanges(cacheDesc)) {
                     // Do not allow any schema changes when keep static cache configuration flag is set.
                     if (log.isDebugEnabled())
                         log.debug("Received schema propose discovery message, but cache is statically configured " +
@@ -2989,7 +2981,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (qry.isDistributedJoins() && qry.getPartitions() != null)
             throw new CacheException("Using both partitions and distributed JOINs is not supported for the same query");
 
-        if (qry.isLocal() && ctx.clientNode() && (cctx == null || cctx.config().getCacheMode() != CacheMode.LOCAL))
+        if (qry.isLocal() && ctx.clientNode())
             throw new CacheException("Execution of local SqlFieldsQuery on client node disallowed.");
 
         return executeQuerySafe(cctx, () -> {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
index 358c82a057d..e58a5eb3ce2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
@@ -22,7 +22,6 @@ import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCluster;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
@@ -57,7 +56,6 @@ import org.apache.ignite.internal.sql.command.SqlKillServiceCommand;
 import org.apache.ignite.internal.sql.command.SqlKillTransactionCommand;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.internal.processors.query.QueryUtils.convert;
 import static org.apache.ignite.internal.processors.query.QueryUtils.isDdlOnSchemaSupported;
 
@@ -221,8 +219,6 @@ public class SqlCommandProcessor {
                 if (typeDesc == null)
                     throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, cmd0.tableName());
 
-                ensureDdlSupported(cacheInfo);
-
                 QueryIndex newIdx = new QueryIndex();
 
                 newIdx.setName(cmd0.indexName());
@@ -255,8 +251,6 @@ public class SqlCommandProcessor {
                     GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(typeDesc.schemaName(),
                         typeDesc.tableName());
 
-                    ensureDdlSupported(cacheInfo);
-
                     fut = ctx.query().dynamicIndexDrop(cacheInfo.name(), cmd0.schemaName(), cmd0.indexName(),
                         cmd0.ifExists());
                 }
@@ -348,17 +342,4 @@ public class SqlCommandProcessor {
                 tx.rollback();
         }
     }
-
-    /**
-     * Check if cache supports DDL statement.
-     *
-     * @param cctxInfo Cache context info.
-     * @throws IgniteSQLException If failed.
-     */
-    protected static void ensureDdlSupported(GridCacheContextInfo<?, ?> cctxInfo) throws IgniteSQLException {
-        if (cctxInfo.config().getCacheMode() == CacheMode.LOCAL) {
-            throw new IgniteSQLException("DDL statements are not supported on LOCAL caches",
-                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
-        }
-    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index a2738ac3b5a..0d239546f26 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -299,7 +299,7 @@ public class VisorCache extends VisorDataTransferObject {
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
         U.writeIgniteUuid(out, dynamicDeploymentId);
-        U.writeEnum(out, mode);
+        out.writeByte(CacheMode.toCode(mode));
         out.writeLong(memorySize);
         out.writeLong(indexesSize);
         out.writeLong(size);
@@ -317,7 +317,7 @@ public class VisorCache extends VisorDataTransferObject {
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         name = U.readString(in);
         dynamicDeploymentId = U.readIgniteUuid(in);
-        mode = CacheMode.fromOrdinal(in.readByte());
+        mode = CacheMode.fromCode(in.readByte());
         memorySize = in.readLong();
         indexesSize = in.readLong();
         size = in.readLong();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
index 6dd91d8bd47..8bcb5de4ab6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
@@ -560,7 +560,7 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject {
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
-        U.writeEnum(out, mode);
+        out.writeByte(CacheMode.toCode(mode));
         out.writeBoolean(sys);
         U.writeMap(out, metrics);
         out.writeObject(minHeapSize);
@@ -592,7 +592,7 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject {
     /** {@inheritDoc} */
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         name = U.readString(in);
-        mode = CacheMode.fromOrdinal(in.readByte());
+        mode = CacheMode.fromCode(in.readByte());
         sys = in.readBoolean();
         metrics = U.readMap(in);
         minHeapSize = (Long)in.readObject();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index c7cf2a99dc0..769a282fe11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -36,7 +36,6 @@ import org.apache.ignite.internal.visor.query.VisorQueryConfiguration;
 import org.apache.ignite.internal.visor.query.VisorQueryEntity;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
-
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactIterable;
 
@@ -551,7 +550,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
         U.writeString(out, grpName);
-        U.writeEnum(out, mode);
+        out.writeByte(CacheMode.toCode(mode));
         U.writeEnum(out, atomicityMode);
         out.writeBoolean(eagerTtl);
         U.writeEnum(out, writeSynchronizationMode);
@@ -599,7 +598,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         name = U.readString(in);
         grpName = U.readString(in);
-        mode = CacheMode.fromOrdinal(in.readByte());
+        mode = CacheMode.fromCode(in.readByte());
         atomicityMode = CacheAtomicityMode.fromOrdinal(in.readByte());
         eagerTtl = in.readBoolean();
         writeSynchronizationMode = CacheWriteSynchronizationMode.fromOrdinal(in.readByte());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
index 854bbd7ad1e..378de7a5e98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
@@ -680,7 +680,7 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
-        U.writeEnum(out, mode);
+        out.writeByte(CacheMode.toCode(mode));
 
         out.writeBoolean(sys);
         out.writeInt(size);
@@ -743,7 +743,7 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
     /** {@inheritDoc} */
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         name = U.readString(in);
-        mode = CacheMode.fromOrdinal(in.readByte());
+        mode = CacheMode.fromCode(in.readByte());
         sys = in.readBoolean();
         size = in.readInt();
         keySize = in.readInt();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java
index 18597f2c7d5..f4078c15601 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorFindAndDeleteGarbageInPersistenceClosure.java
@@ -247,7 +247,7 @@ public class VisorFindAndDeleteGarbageInPersistenceClosure implements IgniteCall
             Collection<CacheGroupContext> groups = ignite.context().cache().cacheGroups();
 
             for (CacheGroupContext grp : groups) {
-                if (!grp.systemCache() && !grp.isLocal())
+                if (!grp.systemCache())
                     grpIds.add(grp.groupId());
             }
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
index 79b468d3e2d..2c77d6ef11c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
@@ -23,7 +23,6 @@ import java.io.ObjectOutput;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
@@ -84,14 +83,14 @@ public class VisorAtomicConfiguration extends VisorDataTransferObject {
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         out.writeInt(seqReserveSize);
-        U.writeEnum(out, cacheMode);
+        out.writeByte(CacheMode.toCode(cacheMode));
         out.writeInt(backups);
     }
 
     /** {@inheritDoc} */
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         seqReserveSize = in.readInt();
-        cacheMode = CacheMode.fromOrdinal(in.readByte());
+        cacheMode = CacheMode.fromCode(in.readByte());
         backups = in.readInt();
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
index 38aa89918ed..634768ac9fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/verify/ValidateIndexesClosure.java
@@ -238,7 +238,7 @@ public class ValidateIndexesClosure implements IgniteCallable<VisorValidateIndex
             Collection<CacheGroupContext> groups = ignite.context().cache().cacheGroups();
 
             for (CacheGroupContext grp : groups) {
-                if (!grp.systemCache() && !grp.isLocal() && grp.affinityNode())
+                if (!grp.systemCache() && grp.affinityNode())
                     grpIds.add(grp.groupId());
             }
         }
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 17490a8e7ae..5675a086625 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -445,7 +445,6 @@ org.apache.ignite.internal.processors.affinity.GridAffinityMessage
 org.apache.ignite.internal.processors.affinity.GridAffinityProcessor$2
 org.apache.ignite.internal.processors.affinity.GridAffinityProcessor$AffinityFuture$1
 org.apache.ignite.internal.processors.affinity.GridAffinityUtils$AffinityJob
-org.apache.ignite.internal.processors.affinity.LocalAffinityFunction
 org.apache.ignite.internal.processors.authentication.IgniteAccessControlException
 org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor$InitialUsersData
 org.apache.ignite.internal.processors.authentication.User
@@ -647,7 +646,6 @@ org.apache.ignite.internal.processors.cache.GridCacheProcessor$7
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$7$1
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$8
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$9
-org.apache.ignite.internal.processors.cache.GridCacheProcessor$LocalAffinityFunction
 org.apache.ignite.internal.processors.cache.GridCacheProxyImpl
 org.apache.ignite.internal.processors.cache.GridCacheReturn
 org.apache.ignite.internal.processors.cache.GridCacheTtlManager$1
@@ -1044,14 +1042,6 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxRemote
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearUnlockRequest
 org.apache.ignite.internal.processors.cache.dr.GridCacheDrExpirationInfo
 org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo
-org.apache.ignite.internal.processors.cache.local.GridLocalCache
-org.apache.ignite.internal.processors.cache.local.GridLocalLockFuture$LockTimeoutObject$1
-org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache
-org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$3
-org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$4
-org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$5
-org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$8
-org.apache.ignite.internal.processors.cache.local.atomic.GridLocalAtomicCache$9
 org.apache.ignite.internal.processors.cache.mvcc.DeadlockProbe
 org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinator
 org.apache.ignite.internal.processors.cache.mvcc.MvccProcessorImpl$2
diff --git a/modules/core/src/test/config/example-cache.xml b/modules/core/src/test/config/example-cache.xml
index fe040bb999a..67f4f78cb61 100644
--- a/modules/core/src/test/config/example-cache.xml
+++ b/modules/core/src/test/config/example-cache.xml
@@ -88,20 +88,6 @@
                     <property name="cacheMode" value="REPLICATED"/>
                     <property name="atomicityMode" value="TRANSACTIONAL"/>
                 </bean>
-
-                  <!-- Local cache example configuration (Atomic mode). -->
-                  <bean parent="cache-template">
-                      <property name="name" value="local"/>
-                      <property name="cacheMode" value="LOCAL"/>
-                      <property name="atomicityMode" value="ATOMIC"/>
-                  </bean>
-
-                <!-- Local cache example configuration (Transactional mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="local_tx"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
             </list>
         </property>
 
diff --git a/modules/core/src/test/config/load/cache-benchmark.xml b/modules/core/src/test/config/load/cache-benchmark.xml
index 4422b539d53..661551c0120 100644
--- a/modules/core/src/test/config/load/cache-benchmark.xml
+++ b/modules/core/src/test/config/load/cache-benchmark.xml
@@ -38,15 +38,6 @@
 
                     <property name="swapEnabled" value="false"/>
 
-                    <property name="writeSynchronizationMode" value="FULL_ASYNC"/>
-                </bean>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="local"/>
-
-                    <property name="cacheMode" value="LOCAL"/>
-
-                    <property name="swapEnabled" value="false"/>
-
                     <property name="writeSynchronizationMode" value="FULL_ASYNC"/>
                 </bean>
             </list>
diff --git a/modules/core/src/test/config/load/cache-client-benchmark.xml b/modules/core/src/test/config/load/cache-client-benchmark.xml
index 41bd97446c7..6ff7e2eca7d 100644
--- a/modules/core/src/test/config/load/cache-client-benchmark.xml
+++ b/modules/core/src/test/config/load/cache-client-benchmark.xml
@@ -34,7 +34,7 @@
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="name" value="local"/>
 
-                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="cacheMode" value="REPLICATED"/>
 
                     <property name="swapEnabled" value="false"/>
 
diff --git a/modules/core/src/test/config/spring-cache-swap.xml b/modules/core/src/test/config/spring-cache-swap.xml
index 6caaf32ef49..0260200e003 100644
--- a/modules/core/src/test/config/spring-cache-swap.xml
+++ b/modules/core/src/test/config/spring-cache-swap.xml
@@ -32,7 +32,7 @@
             <bean class="org.apache.ignite.configuration.CacheConfiguration">
                 <property name="name" value="test-cache"/>
 
-                <property name="cacheMode" value="LOCAL"/>
+                <property name="cacheMode" value="REPLICATED"/>
 
                 <property name="swapEnabled" value="true"/>
 
diff --git a/modules/core/src/test/config/websession/example-cache-base.xml b/modules/core/src/test/config/websession/example-cache-base.xml
index 8777528f794..4f9cf328ed8 100644
--- a/modules/core/src/test/config/websession/example-cache-base.xml
+++ b/modules/core/src/test/config/websession/example-cache-base.xml
@@ -96,20 +96,6 @@
                     <property name="cacheMode" value="REPLICATED"/>
                     <property name="atomicityMode" value="TRANSACTIONAL"/>
                 </bean>
-
-                <!-- Local cache example configuration (Atomic mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="local"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="ATOMIC"/>
-                </bean>
-
-                <!-- Local cache example configuration (Transactional mode). -->
-                <bean parent="cache-template">
-                    <property name="name" value="local_tx"/>
-                    <property name="cacheMode" value="LOCAL"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                </bean>
             </list>
         </property>
 
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/local/LocalAffinityFunctionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/local/LocalAffinityFunctionTest.java
deleted file mode 100755
index 0b62c330f95..00000000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/local/LocalAffinityFunctionTest.java
+++ /dev/null
@@ -1,69 +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.affinity.local;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.affinity.LocalAffinityFunction;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Test;
-
-/**
- * Test for local affinity function.
- */
-public class LocalAffinityFunctionTest extends GridCommonAbstractTest {
-    /** */
-    private static final int NODE_CNT = 1;
-
-    /** */
-    private static final String CACHE1 = "cache1";
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
-
-        ccfg.setBackups(1);
-        ccfg.setName(CACHE1);
-        ccfg.setCacheMode(CacheMode.LOCAL);
-        ccfg.setAffinity(new RendezvousAffinityFunction());
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-        startGrids(NODE_CNT);
-    }
-
-    /** */
-    @Test
-    public void testWronglySetAffinityFunctionForLocalCache() {
-        Ignite node = ignite(NODE_CNT - 1);
-
-        CacheConfiguration ccf = node.cache(CACHE1).getConfiguration(CacheConfiguration.class);
-
-        assertEquals(LocalAffinityFunction.class, ccf.getAffinity().getClass());
-    }
-}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheHashMapPutAllWarningsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheHashMapPutAllWarningsTest.java
index e518a4d297c..c64627a216e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheHashMapPutAllWarningsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheHashMapPutAllWarningsTest.java
@@ -23,15 +23,9 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.TreeSet;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.EntryProcessorResult;
-import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -42,7 +36,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
-import org.junit.Assume;
 import org.junit.Test;
 
 /**
@@ -148,55 +141,6 @@ public class GridCacheHashMapPutAllWarningsTest extends GridCommonAbstractTest {
         }
     }
 
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testHashMapInvokeAllLocal() throws Exception {
-        Assume.assumeFalse( "Local transactional caches not supported by MVCC",
-            IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_FORCE_MVCC_MODE_IN_TESTS, false));
-
-        List<String> messages = Collections.synchronizedList(new ArrayList<>());
-
-        testLog = new ListeningTestLogger(log());
-
-        testLog.registerListener((s) -> {
-            if (s.contains("deadlock"))
-                messages.add(s);
-        });
-
-        Ignite ignite = startGrid(0);
-
-        IgniteCache<Integer, String> c = ignite.getOrCreateCache(new CacheConfiguration<Integer, String>("invoke")
-            .setCacheMode(CacheMode.LOCAL).setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL));
-
-        c.put(1, "foo");
-        c.put(2, "bar");
-
-        Map<Integer, EntryProcessorResult<String>> result = c.invokeAll(new HashSet<>(Arrays.asList(1, 2)),
-            new EntryProcessor<Integer, String, String>() {
-                @Override public String process(MutableEntry entry, Object... arguments) throws EntryProcessorException {
-                    String newVal = entry.getValue() + "2";
-
-                    entry.setValue(newVal);
-
-                    return newVal;
-                }
-            });
-
-        assertEquals(2, result.size());
-        assertEquals("bar2", c.get(2));
-
-        int found = 0;
-
-        for (String message : messages) {
-            if (message.contains("Unordered collection java.util.HashSet is used for invokeAll operation on cache invoke. "))
-                found++;
-        }
-
-        assertEquals(1, found);
-    }
-
     /**
      * @throws Exception If failed.
      */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
index 158188242d5..53b080d42a4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
@@ -232,57 +232,6 @@ public class CacheClientStoreSelfTest extends GridCommonAbstractTest {
         cache.invokeAll(F.asSet(0, 1), new EP());
     }
 
-    /**
-     * Load cache created on client as LOCAL and see if it only loaded on client
-     *
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testLocalLoadClient() throws Exception {
-        cacheMode = CacheMode.LOCAL;
-        factory = new Factory3();
-
-        startGrids(2);
-
-        Ignite client = startClientGrid("client-1");
-
-        IgniteCache<Object, Object> cache = client.cache(CACHE_NAME);
-
-        cache.loadCache(null);
-
-        assertEquals(10, cache.localSize(CachePeekMode.ALL));
-
-        assertEquals(0, grid(0).cache(CACHE_NAME).localSize(CachePeekMode.ALL));
-        assertEquals(0, grid(1).cache(CACHE_NAME).localSize(CachePeekMode.ALL));
-
-        assert loadedFromClient;
-    }
-
-    /**
-     * Load cache from server that created on client as LOCAL and see if it only loaded on server
-     *
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testLocalLoadServer() throws Exception {
-        cacheMode = CacheMode.LOCAL;
-        factory = new Factory3();
-
-        startGrids(2);
-
-        Ignite client = startClientGrid("client-1");
-
-        IgniteCache cache = grid(0).cache(CACHE_NAME);
-
-        cache.loadCache(null);
-
-        assertEquals(10, cache.localSize(CachePeekMode.ALL));
-        assertEquals(0, grid(1).cache(CACHE_NAME).localSize(CachePeekMode.ALL));
-        assertEquals(0, client.cache(CACHE_NAME).localSize(CachePeekMode.ALL));
-
-        assert !loadedFromClient : "Loaded data from client!";
-    }
-
     /**
      * Load cache created on client as REPLICATED and see if it only loaded on servers
      */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java
index 69a19f47816..a9b81364449 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java
@@ -25,11 +25,9 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
... 15053 lines suppressed ...