# ignite-51 merge from sprint-2

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

Branch: refs/heads/ignite-410
Commit: e59ea49803687cf6bd96ae93a34a9a13377ad736
Parents: 3c1eeaf 1ef545a
Author: sboikov <sboi...@gridgain.com>
Authored: Tue Mar 10 09:59:38 2015 +0300
Committer: sboikov <sboi...@gridgain.com>
Committed: Tue Mar 10 09:59:38 2015 +0300

----------------------------------------------------------------------
 DISCLAIMER.txt                                  |  15 +
 README.txt                                      |  25 +
 RELEASE_NOTES.txt                               |  18 +
 assembly/release-base.xml                       |  20 +-
 assembly/release-fabric.xml                     |   5 -
 bin/ignitevisorcmd.bat                          |   7 +-
 bin/ignitevisorcmd.sh                           |   7 +-
 docs/ignite_readme.md                           | 100 ----
 docs/ignite_readme.pdf                          | Bin 77136 -> 0 bytes
 docs/release_notes.md                           |  16 -
 docs/release_notes.pdf                          | Bin 33174 -> 0 bytes
 .../ComputeFibonacciContinuationExample.java    |  12 +-
 .../examples/datagrid/CacheApiExample.java      |   2 +-
 .../examples/ScalarContinuationExample.scala    |  12 +-
 .../client/suite/IgniteClientTestSuite.java     |   3 +-
 .../apache/ignite/IgniteSystemProperties.java   |  20 -
 .../org/apache/ignite/cache/CacheManager.java   |   1 -
 .../internal/ComputeTaskInternalFuture.java     |  44 +-
 .../ignite/internal/GridJobContextImpl.java     |   6 +-
 .../ignite/internal/GridJobSessionImpl.java     |   2 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |  26 -
 .../ignite/internal/GridTaskSessionImpl.java    |   2 +-
 .../ignite/internal/IgniteInternalFuture.java   |  79 +--
 .../internal/client/GridClientFuture.java       |   9 +-
 .../client/impl/GridClientDataImpl.java         |   2 +-
 .../client/impl/GridClientFutureAdapter.java    |  26 +-
 .../connection/GridClientNioTcpConnection.java  |   5 +-
 .../impl/GridTcpRouterNioListenerAdapter.java   |   2 +-
 .../internal/cluster/IgniteClusterImpl.java     |   9 +-
 .../internal/executor/GridExecutorService.java  |   2 +-
 .../igfs/common/IgfsControlResponse.java        |   1 -
 .../managers/communication/GridIoManager.java   |  43 +-
 .../discovery/GridDiscoveryManager.java         |  10 +-
 .../eventstorage/GridEventStorageManager.java   |   6 +-
 .../affinity/GridAffinityAssignmentCache.java   |  17 +-
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../processors/cache/GridCacheAdapter.java      |  87 ++--
 .../cache/GridCacheAffinityManager.java         |   2 +-
 .../cache/GridCacheDeploymentManager.java       |   2 +-
 .../cache/GridCacheEvictionManager.java         |  27 +-
 .../processors/cache/GridCacheGateway.java      |   6 +
 .../processors/cache/GridCacheIoManager.java    |  77 +--
 .../processors/cache/GridCacheMapEntry.java     |   3 +-
 .../processors/cache/GridCacheMessage.java      |   7 -
 .../cache/GridCacheMultiTxFuture.java           |  54 +-
 .../processors/cache/GridCacheMvcc.java         |   3 +-
 .../processors/cache/GridCacheMvccManager.java  |  38 +-
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../cache/GridCachePreloaderAdapter.java        |   4 +-
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../cache/GridCacheProjectionImpl.java          |  12 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 .../processors/cache/GridCacheUtils.java        |  11 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  25 +-
 .../distributed/GridCacheTxFinishSync.java      |   2 +-
 .../GridDistributedCacheAdapter.java            |   4 +-
 .../GridDistributedTxRemoteAdapter.java         |   6 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |  22 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   9 +-
 .../distributed/dht/GridDhtCacheEntry.java      |  14 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |  43 +-
 .../distributed/dht/GridDhtFinishedFuture.java  |  22 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  45 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   4 +-
 .../distributed/dht/GridDhtLockFuture.java      |  35 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  35 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  38 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   4 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  16 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  46 +-
 .../dht/GridPartitionedGetFuture.java           |  42 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  15 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  23 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  37 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  30 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  43 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |  44 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   9 +-
 .../GridDhtPartitionsExchangeFuture.java        |  48 +-
 .../dht/preloader/GridDhtPreloader.java         |  22 +-
 .../distributed/near/GridNearAtomicCache.java   |   2 +-
 .../distributed/near/GridNearCacheAdapter.java  |   2 +-
 .../distributed/near/GridNearGetFuture.java     |  51 +-
 .../distributed/near/GridNearLockFuture.java    |  45 +-
 .../near/GridNearTransactionalCache.java        |   2 +-
 .../near/GridNearTxFinishFuture.java            |  32 +-
 .../cache/distributed/near/GridNearTxLocal.java |  83 +--
 .../near/GridNearTxPrepareFuture.java           |  48 +-
 .../processors/cache/local/GridLocalCache.java  |   2 +-
 .../cache/local/GridLocalLockFuture.java        |  23 +-
 .../processors/cache/local/GridLocalTx.java     |  10 +-
 .../cache/local/GridLocalTxFuture.java          |  45 +-
 .../local/atomic/GridLocalAtomicCache.java      |   8 +-
 .../GridCacheDistributedFieldsQueryFuture.java  |  13 +-
 .../query/GridCacheDistributedQueryFuture.java  |  11 -
 .../query/GridCacheDistributedQueryManager.java |   4 +-
 .../query/GridCacheFieldsQueryErrorFuture.java  |  53 --
 .../query/GridCacheLocalFieldsQueryFuture.java  |  13 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |  15 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |   5 +-
 .../query/GridCacheQueryFutureAdapter.java      |  14 +-
 .../cache/query/GridCacheQueryManager.java      |  11 +-
 .../cache/transactions/IgniteTxAdapter.java     |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |  25 +-
 .../transactions/IgniteTxLocalAdapter.java      | 342 ++++++------
 .../cache/transactions/IgniteTxManager.java     |  18 +-
 .../transactions/TransactionProxyImpl.java      |   2 +-
 .../closure/GridClosureProcessor.java           |  38 +-
 .../continuous/GridContinuousProcessor.java     |  36 +-
 .../dataload/GridDataLoaderFuture.java          |  16 -
 .../dataload/GridDataLoaderProcessor.java       |   2 +-
 .../dataload/IgniteDataLoaderImpl.java          |  35 +-
 .../GridCacheAtomicSequenceImpl.java            |   4 +-
 .../processors/hadoop/HadoopNoopProcessor.java  |   2 +-
 .../processors/igfs/IgfsDataManager.java        |  36 +-
 .../processors/igfs/IgfsDeleteWorker.java       |   2 +-
 .../internal/processors/igfs/IgfsImpl.java      |  12 +-
 .../processors/igfs/IgfsInputStreamImpl.java    |   2 +-
 .../processors/igfs/IgfsIpcHandler.java         |   4 +-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../internal/processors/igfs/IgfsServer.java    |   3 +-
 .../processors/job/GridJobProcessor.java        |   4 +-
 .../processors/query/GridQueryProcessor.java    |   6 +-
 .../processors/resource/GridResourceUtils.java  |   4 +-
 .../processors/rest/GridRestProcessor.java      |  14 +-
 .../handlers/cache/GridCacheCommandHandler.java |   6 +-
 .../cache/GridCacheQueryCommandHandler.java     |   4 +-
 .../DataStructuresCommandHandler.java           |   4 +-
 .../handlers/task/GridTaskCommandHandler.java   |   6 +-
 .../top/GridTopologyCommandHandler.java         |   4 +-
 .../version/GridVersionCommandHandler.java      |   2 +-
 .../tcp/GridTcpMemcachedNioListener.java        |   6 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |   6 +-
 .../service/GridServiceDeploymentFuture.java    |   9 +-
 .../service/GridServiceProcessor.java           |  10 +-
 .../GridStreamerStageExecutionFuture.java       |  32 +-
 .../processors/streamer/IgniteStreamerImpl.java |  23 +-
 .../internal/util/GridSerializableFuture.java   |  28 -
 .../ignite/internal/util/GridThreadLocal.java   | 175 -------
 .../ignite/internal/util/GridThreadLocalEx.java | 210 --------
 .../ignite/internal/util/IgniteUtils.java       |  19 +-
 .../util/future/GridCompoundFuture.java         |  52 +-
 .../util/future/GridCompoundIdentityFuture.java |  18 +-
 .../util/future/GridEmbeddedFuture.java         |  77 +--
 .../util/future/GridFinishedFuture.java         | 158 ++----
 .../util/future/GridFinishedFutureEx.java       | 197 -------
 .../internal/util/future/GridFutureAdapter.java | 365 ++++---------
 .../util/future/GridFutureAdapterEx.java        | 517 -------------------
 .../util/future/GridFutureChainListener.java    |  18 +-
 .../util/future/IgniteFinishedFutureImpl.java   |  27 +-
 .../util/future/IgniteFinishedFutureImplEx.java |  30 --
 .../internal/util/future/IgniteFutureImpl.java  |  31 +-
 .../internal/util/io/GridFilenameUtils.java     |   2 +-
 .../ignite/internal/util/lang/GridFunc.java     |  90 +---
 .../internal/util/lang/GridPlainFuture.java     |  79 ---
 .../util/lang/GridPlainFutureAdapter.java       | 299 -----------
 .../util/nio/GridNioEmbeddedFuture.java         |  12 +-
 .../util/nio/GridNioFinishedFuture.java         |  77 +--
 .../ignite/internal/util/nio/GridNioFuture.java |  84 +--
 .../internal/util/nio/GridNioFutureImpl.java    | 282 +---------
 .../ignite/internal/util/nio/GridNioServer.java |   7 +-
 .../util/nio/GridTcpNioCommunicationClient.java |  18 +-
 .../ignite/internal/util/worker/GridWorker.java |  27 -
 .../internal/util/worker/GridWorkerFuture.java  |  20 -
 .../visor/cache/VisorCacheClearTask.java        |   2 +-
 .../org/apache/ignite/lang/IgniteFuture.java    |  67 +--
 .../lang/IgniteFutureCancelledException.java    |   3 -
 .../lang/IgniteFutureTimeoutException.java      |   3 -
 .../communication/tcp/TcpCommunicationSpi.java  |  32 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   2 +-
 .../internal/GridMultipleJobsSelfTest.java      |   2 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |   2 +-
 .../internal/GridTaskListenerSelfTest.java      |   2 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |   3 +-
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |   7 +-
 .../GridCacheFinishPartitionsSelfTest.java      |   6 +-
 .../GridCachePartitionedLocalStoreSelfTest.java |   7 -
 ...chePartitionedOffHeapLocalStoreSelfTest.java |   7 -
 .../cache/GridCachePutAllFailoverSelfTest.java  |   6 +-
 .../GridCacheReferenceCleanupSelfTest.java      |   2 +-
 .../GridCacheReplicatedLocalStoreSelfTest.java  |   7 -
 ...heReplicatedUsersAffinityMapperSelfTest.java |   7 +-
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |   7 -
 .../GridCacheTxUsersAffinityMapperSelfTest.java |   7 +-
 .../distributed/GridCacheEventAbstractTest.java |  17 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  26 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |  18 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |  42 +-
 .../util/future/GridCompoundFutureSelfTest.java |  30 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |  13 +-
 .../util/future/GridFinishedFutureSelfTest.java | 103 ----
 .../util/future/GridFutureAdapterSelfTest.java  | 115 +----
 .../future/GridFutureListenPerformanceTest.java |  22 +-
 .../util/future/IgniteFutureImplTest.java       |  99 +---
 .../util/future/nio/GridNioFutureSelfTest.java  |   8 +-
 .../lang/GridFutureListenPerformanceTest.java   |   2 +-
 .../loadtests/colocation/GridTestMain.java      |   2 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |   2 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   2 +-
 .../mergesort/GridMergeSortLoadTask.java        |   2 +-
 .../ignite/messaging/GridMessagingSelfTest.java |  12 +-
 .../GridCacheStoreValueBytesTest.java           |   4 +-
 .../testsuites/IgniteLangSelfTestSuite.java     |   1 -
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |   4 +-
 .../hadoop/igfs/HadoopIgfsFuture.java           |   4 +-
 .../hadoop/igfs/HadoopIgfsInProc.java           |   9 +-
 .../hadoop/igfs/HadoopIgfsInputStream.java      |   6 +-
 .../processors/hadoop/igfs/HadoopIgfsIo.java    |   6 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |   5 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |  50 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |  20 +-
 .../proto/HadoopProtocolJobStatusTask.java      |   2 +-
 .../hadoop/shuffle/HadoopShuffle.java           |   4 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |  16 +-
 .../external/HadoopExternalTaskExecutor.java    |  28 +-
 .../child/HadoopChildProcessRunner.java         |  10 +-
 .../HadoopExternalCommunication.java            |   7 +-
 .../HadoopTcpNioCommunicationClient.java        |  12 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   8 +-
 .../cache/jta/GridCacheXAResource.java          |   9 +-
 .../processors/schedule/ScheduleFutureImpl.java | 205 +-------
 .../schedule/GridScheduleSelfTest.java          |   4 +-
 .../ignite/visor/commands/VisorConsole.scala    |  61 ++-
 .../visor/commands/ack/VisorAckCommand.scala    |   4 +-
 .../visor/commands/gc/VisorGcCommand.scala      |   2 -
 .../visor/commands/ping/VisorPingCommand.scala  |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  34 +-
 229 files changed, 1505 insertions(+), 5428 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
index d09438c,50221c1..d71c728
--- 
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
@@@ -305,9 -304,9 +305,9 @@@ public class GridAffinityProcessor exte
                  cctx.config().getAffinity(),
                  cctx.config().getAffinityMapper(),
                  new GridAffinityAssignment(topVer, 
cctx.affinity().assignments(topVer)),
 -                cctx.portableEnabled());
 +                cctx.cacheObjectContext());
  
-             IgniteInternalFuture<AffinityInfo> old = affMap.putIfAbsent(key, 
new GridFinishedFuture<>(ctx, info));
+             IgniteInternalFuture<AffinityInfo> old = affMap.putIfAbsent(key, 
new GridFinishedFuture<>(info));
  
              if (old != null)
                  info = old.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 63dc779,fe95935..c5707ae
--- 
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
@@@ -1754,7 -1915,7 +1754,7 @@@ public abstract class GridCacheAdapter<
              });
          }
  
-         return new GridFinishedFuture<>(ctx.kernalContext(), 
Collections.<KeyCacheObject, CacheObject>emptyMap());
 -        return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
++        return new GridFinishedFuture<>(Collections.<KeyCacheObject, 
CacheObject>emptyMap());
      }
  
      /**
@@@ -2068,17 -2218,16 +2068,16 @@@
                  }
  
                  if (!skipVals && misses != null && readThrough && 
ctx.readThrough()) {
 -                    final Map<K, GridCacheVersion> loadKeys = misses;
 +                    final Map<KeyCacheObject, GridCacheVersion> loadKeys = 
misses;
  
 -                    final IgniteTxLocalAdapter<K, V> tx0 = tx;
 +                    final IgniteTxLocalAdapter tx0 = tx;
  
 -                    final Collection<K> loaded = new HashSet<>();
 +                    final Collection<KeyCacheObject> loaded = new HashSet<>();
  
 -                    return new GridEmbeddedFuture<>(
 -                        ctx.closures().callLocalSafe(ctx.projectSafe(new 
GPC<Map<K, V>>() {
 -                            @Override public Map<K, V> call() throws 
Exception {
 -                                ctx.store().loadAllFromStore(null/*tx*/, 
loadKeys.keySet(), new CI2<K, V>() {
 +                    return new GridEmbeddedFuture(
-                         ctx.kernalContext(),
 +                        ctx.closures().callLocalSafe(ctx.projectSafe(new 
GPC<Map<K1, V1>>() {
 +                            @Override public Map<K1, V1> call() throws 
Exception {
 +                                ctx.store().loadAllFromStore(null/*tx*/, 
loadKeys.keySet(), new CI2<KeyCacheObject, Object>() {
                                      /** New version for all new entries. */
                                      private GridCacheVersion nextVer;
  
@@@ -2163,10 -2303,10 +2162,10 @@@
                          new C2<Map<K, V>, Exception, 
IgniteInternalFuture<Map<K, V>>>() {
                              @Override public IgniteInternalFuture<Map<K, V>> 
apply(Map<K, V> map, Exception e) {
                                  if (e != null)
-                                     return new 
GridFinishedFuture<>(ctx.kernalContext(), e);
+                                     return new GridFinishedFuture<>(e);
  
                                  if (tx0 == null || (!tx0.implicit() && 
tx0.isolation() == READ_COMMITTED)) {
 -                                    Collection<K> notFound = new 
HashSet<>(loadKeys.keySet());
 +                                    Collection<KeyCacheObject> notFound = new 
HashSet<>(loadKeys.keySet());
  
                                      notFound.removeAll(loaded);
  
@@@ -2380,10 -2521,10 +2379,10 @@@
      }
  
      /** {@inheritDoc} */
 -    @Override public IgniteInternalFuture<?> putAllConflictAsync(final Map<? 
extends K, GridCacheDrInfo<V>> drMap)
 +    @Override public IgniteInternalFuture<?> putAllConflictAsync(final 
Map<KeyCacheObject, GridCacheDrInfo> drMap)
          throws IgniteCheckedException {
          if (F.isEmpty(drMap))
-             return new GridFinishedFuture<Object>(ctx.kernalContext());
+             return new GridFinishedFuture<Object>();
  
          ctx.dr().onReceiveCacheEntriesReceived(drMap.size());
  
@@@ -2978,9 -3127,9 +2977,9 @@@
  
      /** {@inheritDoc} */
      @Override public IgniteInternalFuture<?> putAllAsync(final Map<? extends 
K, ? extends V> m,
 -        @Nullable final IgnitePredicate<Cache.Entry<K, V>>... filter) {
 +        @Nullable final CacheEntryPredicate... filter) {
          if (F.isEmpty(m))
-             return new GridFinishedFuture<Object>(ctx.kernalContext());
+             return new GridFinishedFuture<Object>();
  
          if (keyCheck)
              validateCacheKeys(m.keySet());
@@@ -3350,16 -3500,11 +3349,16 @@@
                          ctx.deploy().registerClass(val);
                  }
                  catch (IgniteCheckedException e) {
-                     return new GridFinishedFuture<>(ctx.kernalContext(), e);
+                     return new GridFinishedFuture<>(e);
                  }
  
 -                return tx.removeAllAsync(ctx, Collections.singletonList(key), 
null, true,
 -                    ctx.vararg(F.<K, V>cacheContainsPeek(val)));
 +                IgniteInternalFuture<GridCacheReturn> fut = 
(IgniteInternalFuture)tx.removeAllAsync(ctx,
 +                        Collections.singletonList(key),
 +                        null,
 +                        true,
 +                        ctx.equalsValArray(val));
 +
 +                return fut;
              }
  
              @Override public String toString() {
@@@ -3388,17 -3530,10 +3387,17 @@@
                          ctx.deploy().registerClass(oldVal);
                  }
                  catch (IgniteCheckedException e) {
-                     return new GridFinishedFuture<>(ctx.kernalContext(), e);
+                     return new GridFinishedFuture<>(e);
                  }
  
 -                return tx.putAllAsync(ctx, F.t(key, newVal), true, null, -1, 
ctx.equalsPeekArray(oldVal));
 +                IgniteInternalFuture<GridCacheReturn> fut = 
(IgniteInternalFuture)tx.putAllAsync(ctx,
 +                    F.t(key, newVal),
 +                    true,
 +                    null,
 +                    -1,
 +                    ctx.equalsValArray(oldVal));
 +
 +                return fut;
              }
  
              @Override public String toString() {
@@@ -3847,10 -4014,10 +3846,10 @@@
                  Collection<ClusterNode> nodes = 
ctx.grid().cluster().forDataNodes(name()).nodes();
  
                  if (nodes.isEmpty())
-                     return new GridFinishedFuture<>(ctx.kernalContext());
+                     return new GridFinishedFuture<>();
  
                  return ctx.closures().callAsyncNoFailover(BROADCAST,
 -                    new LoadKeysCallable<>(ctx.name(), keys0, true, plc),
 +                    new LoadKeysCallable<>(ctx.name(), keys, true, plc),
                      nodes,
                      true);
              }

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 0d5be6f,0a9344e..5f3fe2a
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@@ -116,9 -117,9 +117,9 @@@ public class GridCacheIoManager extend
                  IgniteInternalFuture<Long> topFut = 
cctx.discovery().topologyFuture(rmtTopVer);
  
                  if (!topFut.isDone()) {
 -                    final IgniteBiInClosure<UUID, GridCacheMessage<K, V>> c0 
= c;
 +                    final IgniteBiInClosure<UUID, GridCacheMessage> c0 = c;
  
-                     topFut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                     topFut.listen(new CI1<IgniteInternalFuture<Long>>() {
                          @Override public void 
apply(IgniteInternalFuture<Long> t) {
                              onMessage0(nodeId, cacheMsg, c0);
                          }
@@@ -282,13 -290,9 +290,9 @@@
       * @param msg Message.
       * @param c Closure.
       */
 -    private void processMessage(UUID nodeId, GridCacheMessage<K, V> msg,
 -        IgniteBiInClosure<UUID, GridCacheMessage<K, V>> c) {
 +    private void processMessage(UUID nodeId, GridCacheMessage msg,
 +        IgniteBiInClosure<UUID, GridCacheMessage> c) {
          try {
-             // Start clean.
-             if (msg.transactional())
-                 CU.resetTxContext(cctx);
- 
              // We will not end up with storing a bunch of new UUIDs
              // in each cache entry, since node ID is stored in NIO session
              // on handshake.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 7d2a5bf,c13d199..5921d0b
--- 
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
@@@ -143,19 -153,20 +143,20 @@@ public abstract class GridCacheMapEntr
       * @param ttl Time to live.
       * @param hdrId Header id.
       */
 -    protected GridCacheMapEntry(
 -        GridCacheContext<K, V> cctx,
 -        K key,
 +    protected GridCacheMapEntry(GridCacheContext<?, ?> cctx,
 +        KeyCacheObject key,
          int hash,
 -        V val,
 -        GridCacheMapEntry<K, V> next,
 +        CacheObject val,
 +        GridCacheMapEntry next,
          long ttl,
 -        int hdrId
 -    ) {
 +        int hdrId)
 +    {
-         log = U.logger(cctx.kernalContext(), logRef, GridCacheMapEntry.class);
+         if (log == null)
+             log = U.logger(cctx.kernalContext(), logRef, 
GridCacheMapEntry.class);
  
 -        if (cctx.portableEnabled())
 -            key = (K)cctx.kernalContext().portable().detachPortable(key);
 +        key = 
(KeyCacheObject)cctx.kernalContext().cacheObjects().prepareForCache(key, cctx);
 +
 +        assert key != null;
  
          this.key = key;
          this.hash = hash;

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
index 3d0179b,60921e7..7b027ad
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
@@@ -36,18 -35,12 +35,12 @@@ public final class GridCacheMultiTxFutu
      /** Logger reference. */
      private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
  
-     /** Transactions to wait for. */
-     private final Set<IgniteInternalTx> txs = new GridLeanSet<>();
+     /** Logger. */
+     private static IgniteLogger log;
  
      /** */
 -    private Set<IgniteInternalTx<K, V>> remainingTxs;
 +    private Set<IgniteInternalTx> remainingTxs;
  
-     /** Logger. */
-     private IgniteLogger log;
- 
      /**
       * @param cctx Cache context.
       */
@@@ -84,8 -52,11 +52,11 @@@
      /**
       * @param tx Transaction to add.
       */
 -    public void addTx(IgniteInternalTx<K, V> tx) {
 +    public void addTx(IgniteInternalTx tx) {
-         txs.add(tx);
+         if (remainingTxs == null)
+             remainingTxs = new GridConcurrentHashSet<>();
+ 
+         remainingTxs.add(tx);
      }
  
      /**
@@@ -98,11 -69,11 +69,11 @@@
              onDone(true);
          }
          else {
-             remainingTxs = new GridConcurrentHashSet<>(txs);
+             assert !remainingTxs.isEmpty();
  
-             for (final IgniteInternalTx tx : txs) {
 -            for (final IgniteInternalTx<K, V> tx : remainingTxs) {
++            for (final IgniteInternalTx tx : remainingTxs) {
                  if (!tx.done()) {
-                     tx.finishFuture().listenAsync(new 
CI1<IgniteInternalFuture<IgniteInternalTx>>() {
+                     tx.finishFuture().listen(new 
CI1<IgniteInternalFuture<IgniteInternalTx>>() {
                          @Override public void 
apply(IgniteInternalFuture<IgniteInternalTx> t) {
                              remainingTxs.remove(tx);
  

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index a920d07,fc586ba..c717d17
--- 
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
@@@ -53,9 -51,9 +51,9 @@@ public class GridCacheMvccManager exten
      private static final int MAX_REMOVED_LOCKS = 10240;
  
      /** Pending locks per thread. */
-     private final GridThreadLocal<Queue<GridCacheMvccCandidate>> pending =
-         new GridThreadLocal<Queue<GridCacheMvccCandidate>>() {
 -    private final ThreadLocal<Queue<GridCacheMvccCandidate<K>>> pending =
 -        new ThreadLocal<Queue<GridCacheMvccCandidate<K>>>() {
 -            @Override protected Queue<GridCacheMvccCandidate<K>> 
initialValue() {
++    private final ThreadLocal<Queue<GridCacheMvccCandidate>> pending =
++        new ThreadLocal<Queue<GridCacheMvccCandidate>>() {
 +            @Override protected Queue<GridCacheMvccCandidate> initialValue() {
                  return new LinkedList<>();
              }
          };
@@@ -936,9 -938,9 +941,9 @@@
       * @return Explicit locks release future.
       */
      public IgniteInternalFuture<?> finishExplicitLocks(long topVer) {
-         GridCompoundFuture<Object, Object> res = new 
GridCompoundFuture<>(cctx.kernalContext());
+         GridCompoundFuture<Object, Object> res = new GridCompoundFuture<>();
  
 -        for (GridCacheExplicitLockSpan<K> span : pendingExplicit.values()) {
 +        for (GridCacheExplicitLockSpan span : pendingExplicit.values()) {
              GridDiscoveryTopologySnapshot snapshot = span.topologySnapshot();
  
              if (snapshot != null && snapshot.topologyVersion() < topVer)
@@@ -1014,8 -1016,8 +1019,9 @@@
  
          finishFuts.add(finishFut);
  
-         finishFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
-             @Override public void apply(IgniteInternalFuture<?> e) {
+         finishFut.listen(new CI1<IgniteInternalFuture<?>>() {
 -            @Override public void apply(IgniteInternalFuture<?> e) {
++            @Override
++            public void apply(IgniteInternalFuture<?> e) {
                  finishFuts.remove(finishFut);
  
                  // This call is required to make sure that the concurrent 
queue
@@@ -1060,18 -1059,7 +1063,7 @@@
           * @param topVer Topology version.
           * @param entries Entries.
           */
 -        FinishLockFuture(Iterable<GridDistributedCacheEntry<K, V>> entries, 
long topVer) {
 +        FinishLockFuture(Iterable<GridDistributedCacheEntry> entries, long 
topVer) {
-             super(cctx.kernalContext(), true);
- 
              assert topVer > 0;
  
              this.topVer = topVer;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 5d83748,f63d4a3..b7e3473
--- 
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
@@@ -140,8 -140,8 +140,8 @@@ public class GridCachePartitionExchange
  
                          try {
                              // Unwind in the order of discovery events.
 -                            for (GridDhtPartitionsExchangeFuture<K, V> f = 
pendingExchangeFuts.poll(); f != null;
 +                            for (GridDhtPartitionsExchangeFuture f = 
pendingExchangeFuts.poll(); f != null;
-                                 f = pendingExchangeFuts.poll())
+                                  f = pendingExchangeFuts.poll())
                                  addFuture(f);
                          }
                          finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index fb85891,c863e75..18f0283
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@@ -111,8 -111,8 +111,8 @@@ public class GridCachePreloaderAdapter<
      }
  
      /** {@inheritDoc} */
 -    @Override public IgniteInternalFuture<Object> request(Collection<? 
extends K> keys, long topVer) {
 +    @Override public IgniteInternalFuture<Object> 
request(Collection<KeyCacheObject> keys, long topVer) {
-         return new GridFinishedFuture<>(cctx.kernalContext());
+         return new GridFinishedFuture<>();
      }
  
      /** {@inheritDoc} */

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
index 95c2c58,20e3c3b..da7f7e4
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
@@@ -573,10 -775,10 +573,10 @@@ public class GridCacheProjectionImpl<K
          A.notNull(key, "key", val, "val");
  
          // Check k-v predicate first.
 -        if (!isAll(key, val, true))
 +        if (!isAll(key, val))
-             return new GridFinishedFuture<>(cctx.kernalContext());
+             return new GridFinishedFuture<>();
  
 -        return cache.putAsync(key, val, entry, ttl, and(filter, false));
 +        return cache.putAsync(key, val, entry, ttl, and(filter));
      }
  
      /** {@inheritDoc} */
@@@ -655,10 -857,10 +655,10 @@@
          A.notNull(key, "key", val, "val");
  
          // Check k-v predicate first.
 -        if (!isAll(key, val, true))
 +        if (!isAll(key, val))
-             return new GridFinishedFuture<>(cctx.kernalContext(), false);
+             return new GridFinishedFuture<>(false);
  
 -        return cache.putxAsync(key, val, entry, ttl, and(filter, false));
 +        return cache.putxAsync(key, val, entry, ttl, and(filter));
      }
  
      /** {@inheritDoc} */
@@@ -721,13 -923,13 +721,13 @@@
  
      /** {@inheritDoc} */
      @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? 
extends V> m,
 -        @Nullable IgnitePredicate<Cache.Entry<K, V>>[] filter) {
 -        m = isAll(m, true);
 +        @Nullable CacheEntryPredicate[] filter) {
 +        m = isAll(m);
  
          if (F.isEmpty(m))
-             return new GridFinishedFuture<>(cctx.kernalContext());
+             return new GridFinishedFuture<>();
  
 -        return cache.putAllAsync(m, and(filter, false));
 +        return cache.putAllAsync(m, and(filter));
      }
  
      /** {@inheritDoc} */
@@@ -967,8 -1168,8 +967,8 @@@
          A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
  
          // Check k-v predicate first.
 -        if (!isAll(key, newVal, true))
 -            return new GridFinishedFuture<>(new GridCacheReturn<V>(false));
 +        if (!isAll(key, newVal))
-             return new GridFinishedFuture<>(cctx.kernalContext(), new 
GridCacheReturn(true, false));
++            return new GridFinishedFuture<>(new GridCacheReturn(false));
  
          return cache.replacexAsync(key, oldVal, newVal);
      }
@@@ -984,9 -1185,9 +984,9 @@@
      }
  
      /** {@inheritDoc} */
 -    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K 
key, V val) {
 -        return !isAll(key, val, true) ? new GridFinishedFuture<>(
 -            new GridCacheReturn<V>(false)) : cache.removexAsync(key, val);
 +    @Override public IgniteInternalFuture<GridCacheReturn> removexAsync(K 
key, V val) {
-         return !isAll(key, val) ? new 
GridFinishedFuture<>(cctx.kernalContext(),
++        return !isAll(key, val) ? new GridFinishedFuture<>(
 +            new GridCacheReturn(true, false)) : cache.removexAsync(key, val);
      }
  
      /** {@inheritDoc} */
@@@ -996,7 -1197,7 +996,7 @@@
  
      /** {@inheritDoc} */
      @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-         return !isAll(key, val) ? new 
GridFinishedFuture<>(cctx.kernalContext(), false) :
 -        return !isAll(key, val, true) ? new GridFinishedFuture<>(false) :
++        return !isAll(key, val) ? new GridFinishedFuture<>(false) :
              cache.removeAsync(key, val);
      }
  

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
index aca2698,7805487..a35b143
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
@@@ -34,15 -35,18 +35,18 @@@ import static org.apache.ignite.interna
  /**
   * Future that fetches affinity assignment from remote cache nodes.
   */
 -public class GridDhtAssignmentFetchFuture<K, V> extends 
GridFutureAdapter<List<List<ClusterNode>>> {
 +public class GridDhtAssignmentFetchFuture extends 
GridFutureAdapter<List<List<ClusterNode>>> {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
      /** Nodes order comparator. */
      private static final Comparator<ClusterNode> CMP = new 
GridNodeOrderComparator();
  
+     /** Logger reference. */
+     private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
+ 
+     /** Logger. */
+     private static IgniteLogger log;
+ 
      /** Cache context. */
 -    private final GridCacheContext<K, V> ctx;
 +    private final GridCacheContext ctx;
  
      /** List of available nodes this future can fetch data from. */
      private Queue<ClusterNode> availableNodes;
@@@ -57,11 -61,12 +61,12 @@@
       * @param ctx Cache context.
       * @param availableNodes Available nodes.
       */
-     public GridDhtAssignmentFetchFuture(GridCacheContext ctx, long topVer, 
Collection<ClusterNode> availableNodes) {
-         super(ctx.kernalContext());
- 
+     public GridDhtAssignmentFetchFuture(
 -        GridCacheContext<K, V> ctx,
++        GridCacheContext ctx,
+         long topVer,
+         Collection<ClusterNode> availableNodes
+     ) {
          this.ctx = ctx;
- 
          this.topVer = topVer;
  
          LinkedList<ClusterNode> tmp = new LinkedList<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 78f2557,e3debe8..398c205
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@@ -627,9 -631,10 +627,9 @@@ public abstract class GridDhtCacheAdapt
                  expiryPlc,
                  req.skipValues());
  
-         fut.listenAsync(new 
CI1<IgniteInternalFuture<Collection<GridCacheEntryInfo>>>() {
 -        fut.listen(new 
CI1<IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>>>() {
 -            @Override public void 
apply(IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> f) {
 -                GridNearGetResponse<K, V> res = new 
GridNearGetResponse<>(ctx.cacheId(),
++        fut.listen(new 
CI1<IgniteInternalFuture<Collection<GridCacheEntryInfo>>>() {
 +            @Override public void 
apply(IgniteInternalFuture<Collection<GridCacheEntryInfo>> f) {
 +                GridNearGetResponse res = new 
GridNearGetResponse(ctx.cacheId(),
                      req.futureId(),
                      req.miniId(),
                      req.version());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index 6226f09,a79e80e..05500e3
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@@ -441,14 -445,18 +441,18 @@@ public class GridDhtCacheEntry extends 
              txFut.init();
  
              if (!txFut.isDone()) {
 -                final ReaderId<K, V> reader0 = reader;
 +                final ReaderId reader0 = reader;
  
-                 txFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                 txFut.listen(new CI1<IgniteInternalFuture<?>>() {
                      @Override public void apply(IgniteInternalFuture<?> f) {
-                         synchronized (this) {
-                             // Release memory.
-                             reader0.resetTxFuture();
-                         }
+                         cctx.kernalContext().closure().runLocalSafe(new 
GridPlainRunnable() {
+                             @Override public void run() {
+                                 synchronized (this) {
+                                     // Release memory.
+                                     reader0.resetTxFuture();
+                                 }
+                             }
+                         });
                      }
                  });
              }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index bc84749,5cff433..bec8c4a
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@@ -37,11 -36,8 +36,8 @@@ import java.util.concurrent.atomic.*
  /**
   *
   */
 -public final class GridDhtGetFuture<K, V> extends 
GridCompoundIdentityFuture<Collection<GridCacheEntryInfo<K, V>>>
 -    implements GridDhtFuture<Collection<GridCacheEntryInfo<K, V>>> {
 +public final class GridDhtGetFuture<K, V> extends 
GridCompoundIdentityFuture<Collection<GridCacheEntryInfo>>
 +    implements GridDhtFuture<Collection<GridCacheEntryInfo>> {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
      /** Logger reference. */
      private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
  
@@@ -76,11 -75,8 +75,8 @@@
      private long topVer;
  
      /** Transaction. */
 -    private IgniteTxLocalEx<K, V> tx;
 +    private IgniteTxLocalEx tx;
  
-     /** Logger. */
-     private IgniteLogger log;
- 
      /** Retries because ownership changed. */
      private Collection<Integer> retries = new GridLeanSet<>();
  
@@@ -208,9 -209,9 +196,9 @@@
          if (!F.isEmpty(fut.invalidPartitions()))
              retries.addAll(fut.invalidPartitions());
  
-         add(new GridEmbeddedFuture<>(cctx.kernalContext(), fut,
+         add(new GridEmbeddedFuture<>(
 -            new IgniteBiClosure<Object, Exception, 
Collection<GridCacheEntryInfo<K, V>>>() {
 -                @Override public Collection<GridCacheEntryInfo<K, V>> 
apply(Object o, Exception e) {
 +            new IgniteBiClosure<Object, Exception, 
Collection<GridCacheEntryInfo>>() {
 +                @Override public Collection<GridCacheEntryInfo> apply(Object 
o, Exception e) {
                      if (e != null) { // Check error first.
                          if (log.isDebugEnabled())
                              log.debug("Failed to request keys from preloader 
[keys=" + keys + ", err=" + e + ']');
@@@ -274,19 -275,17 +262,19 @@@
       * @return Future for local get.
       */
      @SuppressWarnings( {"unchecked", "IfMayBeConditional"})
 -    private IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> 
getAsync(final LinkedHashMap<? extends K, Boolean> keys) {
 +    private IgniteInternalFuture<Collection<GridCacheEntryInfo>> getAsync(
 +        final LinkedHashMap<KeyCacheObject, Boolean> keys)
 +    {
          if (F.isEmpty(keys))
-             return new 
GridFinishedFuture<Collection<GridCacheEntryInfo>>(cctx.kernalContext(),
 -            return new GridFinishedFuture<Collection<GridCacheEntryInfo<K, 
V>>>(
 -                Collections.<GridCacheEntryInfo<K, V>>emptyList());
++            return new GridFinishedFuture<Collection<GridCacheEntryInfo>>(
 +                Collections.<GridCacheEntryInfo>emptyList());
  
 -        final Collection<GridCacheEntryInfo<K, V>> infos = new LinkedList<>();
 +        final Collection<GridCacheEntryInfo> infos = new LinkedList<>();
  
-         String taskName0 = ctx.job().currentTaskName();
+         String taskName0 = cctx.kernalContext().job().currentTaskName();
  
          if (taskName0 == null)
-             taskName0 = ctx.task().resolveTaskName(taskNameHash);
+             taskName0 = 
cctx.kernalContext().task().resolveTaskName(taskNameHash);
  
          final String taskName = taskName0;
  
@@@ -403,13 -402,13 +391,13 @@@
                              }
                          }
                      }
-                 },
-                 cctx.kernalContext());
+                 }
+             );
          }
  
-         return new GridEmbeddedFuture<>(cctx.kernalContext(), fut,
+         return new GridEmbeddedFuture<>(
 -            new C2<Map<K, V>, Exception, Collection<GridCacheEntryInfo<K, 
V>>>() {
 -                @Override public Collection<GridCacheEntryInfo<K, V>> 
apply(Map<K, V> map, Exception e) {
 +            new C2<Map<KeyCacheObject, CacheObject>, Exception, 
Collection<GridCacheEntryInfo>>() {
 +                @Override public Collection<GridCacheEntryInfo> 
apply(Map<KeyCacheObject, CacheObject> map, Exception e) {
                      if (e != null) {
                          onDone(e);
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 2d049cc,bef6468..8e14481
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@@ -48,10 -47,7 +47,7 @@@ import static org.apache.ignite.interna
   * Cache lock future.
   */
  public final class GridDhtLockFuture<K, V> extends 
GridCompoundIdentityFuture<Boolean>
 -    implements GridCacheMvccFuture<K, V, Boolean>, GridDhtFuture<Boolean>, 
GridCacheMappedVersion {
 +    implements GridCacheMvccFuture<Boolean>, GridDhtFuture<Boolean>, 
GridCacheMappedVersion {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
      /** Logger reference. */
      private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
  
@@@ -102,15 -101,11 +101,11 @@@
      /** Lock timeout. */
      private long timeout;
  
-     /** Logger. */
-     @GridToStringExclude
-     private IgniteLogger log;
- 
      /** Filter. */
 -    private IgnitePredicate<Cache.Entry<K, V>>[] filter;
 +    private CacheEntryPredicate[] filter;
  
      /** Transaction. */
 -    private GridDhtTxLocalAdapter<K, V> tx;
 +    private GridDhtTxLocalAdapter tx;
  
      /** All replies flag. */
      private AtomicBoolean mapped = new AtomicBoolean(false);
@@@ -976,22 -963,13 +962,13 @@@
  
          /** DHT mapping. */
          @GridToStringInclude
 -        private List<GridDhtCacheEntry<K, V>> dhtMapping;
 +        private List<GridDhtCacheEntry> dhtMapping;
  
          /**
-          * Empty constructor required for {@link Externalizable}.
-          */
-         public MiniFuture() {
-             // No-op.
-         }
- 
-         /**
           * @param node Node.
           * @param dhtMapping Mapping.
           */
 -        MiniFuture(ClusterNode node, List<GridDhtCacheEntry<K, V>> 
dhtMapping) {
 +        MiniFuture(ClusterNode node, List<GridDhtCacheEntry> dhtMapping) {
-             super(cctx.kernalContext());
- 
              assert node != null;
  
              this.node = node;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 04b6f4d,eb5fed7..2fa5112
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@@ -591,11 -589,11 +591,11 @@@ public abstract class GridDhtTransactio
          boolean retval,
          TransactionIsolation isolation,
          long accessTtl,
 -        IgnitePredicate<Cache.Entry<K, V>>[] filter) {
 +        CacheEntryPredicate[] filter) {
          if (keys == null || keys.isEmpty())
-             return new GridDhtFinishedFuture<>(ctx.kernalContext(), true);
+             return new GridDhtFinishedFuture<>(true);
  
 -        GridDhtTxLocalAdapter<K, V> tx = (GridDhtTxLocalAdapter<K, V>)txx;
 +        GridDhtTxLocalAdapter tx = (GridDhtTxLocalAdapter)txx;
  
          assert tx != null;
  
@@@ -683,18 -684,18 +683,18 @@@
          }
  
          if (keyFut == null)
-             keyFut = new GridFinishedFutureEx<>();
+             keyFut = new GridFinishedFuture<>();
  
-         return new GridEmbeddedFuture<>(true, keyFut,
+         return new GridEmbeddedFuture<>(keyFut,
 -            new C2<Object, Exception, 
IgniteInternalFuture<GridNearLockResponse<K,V>>>() {
 -                @Override public IgniteInternalFuture<GridNearLockResponse<K, 
V>> apply(Object o, Exception exx) {
 +            new C2<Object, Exception, 
IgniteInternalFuture<GridNearLockResponse>>() {
 +                @Override public IgniteInternalFuture<GridNearLockResponse> 
apply(Object o, Exception exx) {
                      if (exx != null)
-                         return new 
GridDhtFinishedFuture<>(ctx.kernalContext(), exx);
+                         return new GridDhtFinishedFuture<>(exx);
  
 -                    IgnitePredicate<Cache.Entry<K, V>>[] filter = filter0;
 +                    CacheEntryPredicate[] filter = filter0;
  
                      // Set message into thread context.
 -                    GridDhtTxLocal<K, V> tx = null;
 +                    GridDhtTxLocal tx = null;
  
                      try {
                          int cnt = keys.size();
@@@ -891,10 -892,8 +891,8 @@@
                              final GridCacheVersion mappedVer = fut.version();
  
                              return new GridDhtEmbeddedFuture<>(
-                                 ctx.kernalContext(),
-                                 fut,
 -                                new C2<Boolean, Exception, 
GridNearLockResponse<K, V>>() {
 -                                    @Override public GridNearLockResponse<K, 
V> apply(Boolean b, Exception e) {
 +                                new C2<Boolean, Exception, 
GridNearLockResponse>() {
 +                                    @Override public GridNearLockResponse 
apply(Boolean b, Exception e) {
                                          if (e != null)
                                              e = U.unwrap(e);
                                          else if (!b)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index d345daf,631e466..cf3b36b
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@@ -427,22 -414,13 +414,13 @@@ public final class GridDhtTxFinishFutur
  
          /** Near mapping. */
          @GridToStringInclude
 -        private GridDistributedTxMapping<K, V> nearMapping;
 +        private GridDistributedTxMapping nearMapping;
  
          /**
-          * Empty constructor required for {@link Externalizable}.
-          */
-         public MiniFuture() {
-             // No-op.
-         }
- 
-         /**
           * @param dhtMapping Mapping.
           * @param nearMapping nearMapping.
           */
 -        MiniFuture(GridDistributedTxMapping<K, V> dhtMapping, 
GridDistributedTxMapping<K, V> nearMapping) {
 +        MiniFuture(GridDistributedTxMapping dhtMapping, 
GridDistributedTxMapping nearMapping) {
-             super(cctx.kernalContext());
- 
              assert dhtMapping == null || nearMapping == null || 
dhtMapping.node() == nearMapping.node();
  
              this.dhtMapping = dhtMapping;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index fd989bb,747686e..2dd9496
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@@ -511,8 -509,8 +511,8 @@@ public class GridDhtTxLocal extends Gri
                  }
              }
              else
-                 prep.listenAsync(new 
CI1<IgniteInternalFuture<IgniteInternalTx>>() {
 -                prep.listen(new CI1<IgniteInternalFuture<IgniteInternalTx<K, 
V>>>() {
 -                    @Override public void 
apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
++                prep.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() 
{
 +                    @Override public void 
apply(IgniteInternalFuture<IgniteInternalTx> f) {
                          try {
                              f.get(); // Check for errors of a parent future.
  
@@@ -599,8 -597,8 +599,8 @@@
          else {
              prepFut.complete();
  
-             prepFut.listenAsync(new 
CI1<IgniteInternalFuture<IgniteInternalTx>>() {
 -            prepFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx<K, 
V>>>() {
 -                @Override public void 
apply(IgniteInternalFuture<IgniteInternalTx<K, V>> f) {
++            prepFut.listen(new CI1<IgniteInternalFuture<IgniteInternalTx>>() {
 +                @Override public void 
apply(IgniteInternalFuture<IgniteInternalTx> f) {
                      try {
                          f.get(); // Check for errors of a parent future.
                      }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 5cca001,6c5d248..d2a658a
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@@ -512,13 -526,13 +512,13 @@@ public abstract class GridDhtTxLocalAda
              checkValid();
          }
          catch (IgniteCheckedException e) {
-             return new GridFinishedFuture<>(cctx.kernalContext(), e);
+             return new GridFinishedFuture<>(e);
          }
  
 -        final GridCacheReturn<V> ret = new GridCacheReturn<>(false);
 +        final GridCacheReturn ret = new GridCacheReturn(localResult(), false);
  
          if (F.isEmpty(entries))
-             return new GridFinishedFuture<>(cctx.kernalContext(), ret);
+             return new GridFinishedFuture<>(ret);
  
          init();
  
@@@ -619,9 -633,9 +619,9 @@@
                  skipped + ']');
  
          if (passedKeys.isEmpty())
-             return new GridFinishedFuture<>(cctx.kernalContext(), ret);
+             return new GridFinishedFuture<>(ret);
  
 -        GridDhtTransactionalCacheAdapter<K, V> dhtCache = cacheCtx.isNear() ? 
cacheCtx.nearTx().dht() : cacheCtx.dhtTx();
 +        GridDhtTransactionalCacheAdapter<?, ?> dhtCache = cacheCtx.isNear() ? 
cacheCtx.nearTx().dht() : cacheCtx.dhtTx();
  
          IgniteInternalFuture<Boolean> fut = 
dhtCache.lockAllAsyncInternal(passedKeys,
              lockTimeout(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index dd14954,90df2c6..a7c706f
--- 
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
@@@ -49,11 -48,8 +48,8 @@@ import static org.apache.ignite.interna
  /**
   *
   */
 -public final class GridDhtTxPrepareFuture<K, V> extends 
GridCompoundIdentityFuture<IgniteInternalTx<K, V>>
 -    implements GridCacheMvccFuture<K, V, IgniteInternalTx<K, V>> {
 +public final class GridDhtTxPrepareFuture<K, V> extends 
GridCompoundIdentityFuture<IgniteInternalTx>
 +    implements GridCacheMvccFuture<IgniteInternalTx> {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
      /** Logger reference. */
      private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
  
@@@ -65,17 -64,14 +64,14 @@@
  
      /** Transaction. */
      @GridToStringExclude
 -    private GridDhtTxLocalAdapter<K, V> tx;
 +    private GridDhtTxLocalAdapter tx;
  
      /** Near mappings. */
 -    private Map<UUID, GridDistributedTxMapping<K, V>> nearMap;
 +    private Map<UUID, GridDistributedTxMapping> nearMap;
  
      /** DHT mappings. */
 -    private Map<UUID, GridDistributedTxMapping<K, V>> dhtMap;
 +    private Map<UUID, GridDistributedTxMapping> dhtMap;
  
-     /** Logger. */
-     private IgniteLogger log;
- 
      /** Error. */
      private AtomicReference<Throwable> err = new AtomicReference<>(null);
  
@@@ -125,16 -121,9 +121,9 @@@
      private volatile boolean locksReady;
  
      /** */
 -    private IgniteInClosure<GridNearTxPrepareResponse<K, V>> completeCb;
 +    private IgniteInClosure<GridNearTxPrepareResponse> completeCb;
  
      /**
-      * Empty constructor required for {@link Externalizable}.
-      */
-     public GridDhtTxPrepareFuture() {
-         // No-op.
-     }
- 
-     /**
       * @param cctx Context.
       * @param tx Transaction.
       * @param nearMiniId Near mini future id.
@@@ -495,11 -487,11 +485,11 @@@
  
              if (!tx.near()) {
                  if 
(tx.markFinalizing(IgniteInternalTx.FinalizationStatus.USER_FINISH)) {
-                     IgniteInternalFuture<IgniteInternalTx> fut =
-                         this.err.get() == null ? tx.commitAsync() : 
tx.rollbackAsync();
+                     IgniteInternalFuture<IgniteInternalTx> fut = 
this.err.get() == null ?
+                         tx.commitAsync() : tx.rollbackAsync();
  
-                     fut.listenAsync(new 
CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
+                     fut.listen(new 
CIX1<IgniteInternalFuture<IgniteInternalTx>>() {
 -                        @Override public void 
applyx(IgniteInternalFuture<IgniteInternalTx> gridCacheTxGridFuture) {
 +                        @Override public void 
applyx(IgniteInternalFuture<IgniteInternalTx> fut) {
                              try {
                                  if (replied.compareAndSet(false, true))
                                      sendPrepareResponse(res);
@@@ -1087,11 -1103,8 +1077,8 @@@
       * Mini-future for get operations. Mini-futures are only waiting on a 
single
       * node as opposed to multiple nodes.
       */
 -    private class MiniFuture extends GridFutureAdapter<IgniteInternalTx<K, 
V>> {
 +    private class MiniFuture extends GridFutureAdapter<IgniteInternalTx> {
          /** */
-         private static final long serialVersionUID = 0L;
- 
-         /** */
          private final IgniteUuid futId = IgniteUuid.randomUuid();
  
          /** Node ID. */
@@@ -1103,23 -1116,18 +1090,18 @@@
  
          /** Near mapping. */
          @GridToStringInclude
 -        private GridDistributedTxMapping<K, V> nearMapping;
 +        private GridDistributedTxMapping nearMapping;
  
          /**
-          * Empty constructor required for {@link Externalizable}.
-          */
-         public MiniFuture() {
-             super(cctx.kernalContext());
-         }
- 
-         /**
           * @param nodeId Node ID.
           * @param dhtMapping Mapping.
           * @param nearMapping nearMapping.
           */
-         MiniFuture(UUID nodeId, GridDistributedTxMapping dhtMapping, 
GridDistributedTxMapping nearMapping) {
-             super(cctx.kernalContext());
- 
+         MiniFuture(
+             UUID nodeId,
 -            GridDistributedTxMapping<K, V> dhtMapping,
 -            GridDistributedTxMapping<K, V> nearMapping
++            GridDistributedTxMapping dhtMapping,
++            GridDistributedTxMapping nearMapping
+         ) {
              assert dhtMapping == null || nearMapping == null || 
dhtMapping.node() == nearMapping.node();
  
              this.nodeId = nodeId;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 603141b,8adf230..cac6e30
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@@ -569,9 -583,7 +549,7 @@@ public class GridPartitionedGetFuture<K
           * @param keys Keys.
           * @param topVer Topology version.
           */
 -        MiniFuture(ClusterNode node, LinkedHashMap<K, Boolean> keys, long 
topVer) {
 +        MiniFuture(ClusterNode node, LinkedHashMap<KeyCacheObject, Boolean> 
keys, long topVer) {
-             super(cctx.kernalContext());
- 
              this.node = node;
              this.keys = keys;
              this.topVer = topVer;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index a5f319e,dd02343..2ec1908
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@@ -979,7 -995,7 +979,7 @@@ public class GridDhtAtomicCache<K, V> e
              if (success) {
                  sendTtlUpdateRequest(expiry);
  
-                 return new GridFinishedFuture<>(ctx.kernalContext(), locVals);
 -                return ctx.wrapCloneMap(new GridFinishedFuture<>(locVals));
++                return new GridFinishedFuture<>(locVals);
              }
          }
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 47262ed,4caf543..449f6ea
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@@ -43,10 -42,8 +42,8 @@@ import static org.apache.ignite.cache.C
  /**
   * DHT atomic cache backup update future.
   */
- public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void> 
implements GridCacheAtomicFuture<Void> {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
 -public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
 -    implements GridCacheAtomicFuture<K, Void> {
++public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
++    implements GridCacheAtomicFuture<Void> {
      /** Logger reference. */
      private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
  
@@@ -103,15 -93,12 +93,13 @@@
       * @param updateRes Update response.
       */
      public GridDhtAtomicUpdateFuture(
 -        GridCacheContext<K, V> cctx,
 -        CI2<GridNearAtomicUpdateRequest<K, V>, 
GridNearAtomicUpdateResponse<K, V>> completionCb,
 +        GridCacheContext cctx,
 +        CI2<GridNearAtomicUpdateRequest,
 +        GridNearAtomicUpdateResponse> completionCb,
          GridCacheVersion writeVer,
 -        GridNearAtomicUpdateRequest<K, V> updateReq,
 -        GridNearAtomicUpdateResponse<K, V> updateRes
 +        GridNearAtomicUpdateRequest updateReq,
 +        GridNearAtomicUpdateResponse updateRes
      ) {
-         super(cctx.kernalContext());
- 
          this.cctx = cctx;
          this.writeVer = writeVer;
  
@@@ -232,11 -222,11 +221,11 @@@
          for (ClusterNode node : dhtNodes) {
              UUID nodeId = node.id();
  
-             if (!nodeId.equals(ctx.localNodeId())) {
+             if (!nodeId.equals(cctx.localNodeId())) {
 -                GridDhtAtomicUpdateRequest<K, V> updateReq = 
mappings.get(nodeId);
 +                GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId);
  
                  if (updateReq == null) {
 -                    updateReq = new GridDhtAtomicUpdateRequest<>(
 +                    updateReq = new GridDhtAtomicUpdateRequest(
                          cctx.cacheId(),
                          nodeId,
                          futVer,
@@@ -282,10 -276,10 +271,10 @@@
          long topVer = updateReq.topologyVersion();
  
          for (UUID nodeId : readers) {
 -            GridDhtAtomicUpdateRequest<K, V> updateReq = mappings.get(nodeId);
 +            GridDhtAtomicUpdateRequest updateReq = mappings.get(nodeId);
  
              if (updateReq == null) {
-                 ClusterNode node = ctx.discovery().node(nodeId);
+                 ClusterNode node = cctx.discovery().node(nodeId);
  
                  // Node left the grid.
                  if (node == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 3b61388,fcbc634..0577c75
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@@ -52,10 -52,8 +52,8 @@@ import static org.apache.ignite.interna
  /**
   * DHT atomic cache near update future.
   */
- public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object> 
implements GridCacheAtomicFuture<Object> {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
 -public class GridNearAtomicUpdateFuture<K, V> extends 
GridFutureAdapter<Object>
 -    implements GridCacheAtomicFuture<K, Object>{
++public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
++    implements GridCacheAtomicFuture<Object>{
      /** Logger reference. */
      private static final AtomicReference<IgniteLogger> logRef = new 
AtomicReference<>();
  
@@@ -786,11 -752,11 +765,11 @@@
          singleNodeId = nodeId;
          singleReq = req;
  
-         if (ctx.localNodeId().equals(nodeId)) {
+         if (cctx.localNodeId().equals(nodeId)) {
              cache.updateAllAsyncInternal(nodeId, req,
 -                new CI2<GridNearAtomicUpdateRequest<K, V>, 
GridNearAtomicUpdateResponse<K, V>>() {
 -                    @Override public void 
apply(GridNearAtomicUpdateRequest<K, V> req,
 -                        GridNearAtomicUpdateResponse<K, V> res) {
 +                new CI2<GridNearAtomicUpdateRequest, 
GridNearAtomicUpdateResponse>() {
 +                    @Override public void apply(GridNearAtomicUpdateRequest 
req,
 +                        GridNearAtomicUpdateResponse res) {
                          assert res.futureVersion().equals(futVer);
  
                          onResult(res.nodeId(), res);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index e1b8963,791fb73..d24d6d8
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@@ -181,12 -167,9 +181,12 @@@ public class GridDhtColocatedCache<K, V
          ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
  
          if (F.isEmpty(keys))
-             return new GridFinishedFuture<>(ctx.kernalContext(), 
Collections.<K, V>emptyMap());
+             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
  
 -        IgniteTxLocalAdapter<K, V> tx = ctx.tm().threadLocalTx();
 +        if (keyCheck)
 +            validateCacheKeys(keys);
 +
 +        IgniteTxLocalAdapter tx = ctx.tm().threadLocalTx();
  
          if (tx != null && !tx.implicit() && !skipTx) {
              return asyncOp(tx, new AsyncOp<Map<K, V>>(keys) {
@@@ -255,8 -232,11 +255,8 @@@
          boolean skipVals
      ) {
          if (keys == null || keys.isEmpty())
-             return new GridFinishedFuture<>(ctx.kernalContext(), 
Collections.<K, V>emptyMap());
+             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
  
 -        if (keyCheck)
 -            validateCacheKeys(keys);
 -
          if (expiryPlc == null)
              expiryPlc = expiryPolicy(null);
  
@@@ -338,7 -325,7 +338,7 @@@
              if (success) {
                  sendTtlUpdateRequest(expiryPlc);
  
-                 return new GridFinishedFuture<>(ctx.kernalContext(), locVals);
 -                return ctx.wrapCloneMap(new GridFinishedFuture<>(locVals));
++                return new GridFinishedFuture<>(locVals);
              }
          }
  
@@@ -772,10 -757,8 +771,8 @@@
                  accessTtl);
  
              return new GridDhtEmbeddedFuture<>(
-                 ctx.kernalContext(),
-                 txFut,
 -                new C2<GridCacheReturn<V>, Exception, Exception>() {
 -                    @Override public Exception apply(GridCacheReturn<V> ret,
 +                new C2<GridCacheReturn, Exception, Exception>() {
 +                    @Override public Exception apply(GridCacheReturn ret,
                          Exception e) {
                          if (e != null)
                              e = U.unwrap(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 0f036c7,3d67e27..10fabb7
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@@ -90,12 -89,8 +89,8 @@@ public final class GridDhtColocatedLock
      /** Lock timeout. */
      private long timeout;
  
-     /** Logger. */
-     @GridToStringExclude
-     private IgniteLogger log;
- 
      /** Filter. */
 -    private IgnitePredicate<Cache.Entry<K, V>>[] filter;
 +    private CacheEntryPredicate[] filter;
  
      /** Transaction. */
      @GridToStringExclude
@@@ -1118,11 -1095,8 +1095,9 @@@
           * @param keys Keys.
           * @param mappings Mappings to proceed.
           */
 -        MiniFuture(ClusterNode node, Collection<K> keys,
 -            Deque<GridNearLockMapping<K, V>> mappings) {
 +        MiniFuture(ClusterNode node,
 +            Collection<KeyCacheObject> keys,
 +            Deque<GridNearLockMapping> mappings) {
-             super(cctx.kernalContext());
- 
              this.node = node;
              this.keys = keys;
              this.mappings = mappings;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index db56ebb,40459af..97ead96
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@@ -59,13 -58,10 +58,10 @@@ public final class GridDhtForceKeysFutu
      private GridCacheContext<K, V> cctx;
  
      /** Topology. */
 -    private GridDhtPartitionTopology<K, V> top;
 +    private GridDhtPartitionTopology top;
  
-     /** Logger. */
-     private IgniteLogger log;
- 
      /** Keys to request. */
 -    private Collection<? extends K> keys;
 +    private Collection<KeyCacheObject> keys;
  
      /** Keys for which local node is no longer primary. */
      private Collection<Integer> invalidParts = new GridLeanSet<>();
@@@ -91,10 -87,11 +87,12 @@@
       * @param keys Keys.
       * @param preloader Preloader.
       */
-     public GridDhtForceKeysFuture(GridCacheContext<K, V> cctx, long topVer, 
Collection<KeyCacheObject> keys,
-         GridDhtPreloader<K, V> preloader) {
-         super(cctx.kernalContext());
- 
+     public GridDhtForceKeysFuture(
+         GridCacheContext<K, V> cctx,
 -        long topVer, Collection<? extends K> keys,
++        long topVer,
++        Collection<KeyCacheObject> keys,
+         GridDhtPreloader<K, V> preloader
+     ) {
          assert topVer != 0 : topVer;
          assert !F.isEmpty(keys) : keys;
  
@@@ -404,9 -383,7 +384,7 @@@
           * @param curTopVer Topology version for this mini-future.
           * @param exc Exclude node list.
           */
 -        MiniFuture(ClusterNode node, Collection<K> keys, int curTopVer, 
Collection<ClusterNode> exc) {
 +        MiniFuture(ClusterNode node, Collection<KeyCacheObject> keys, int 
curTopVer, Collection<ClusterNode> exc) {
-             super(cctx.kernalContext());
- 
              assert node != null;
              assert curTopVer > 0;
              assert exc != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 1cfd3a3,5e5555d..819797a
--- 
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,11 -46,8 +46,8 @@@ import static org.apache.ignite.interna
  /**
   * Future for exchanging partition maps.
   */
 -public class GridDhtPartitionsExchangeFuture<K, V> extends 
GridFutureAdapter<Long>
 -    implements Comparable<GridDhtPartitionsExchangeFuture<K, V>>, 
GridDhtTopologyFuture {
 +public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<Long>
 +    implements Comparable<GridDhtPartitionsExchangeFuture>, 
GridDhtTopologyFuture {
-     /** */
-     private static final long serialVersionUID = 0L;
- 
      /** Dummy flag. */
      private final boolean dummy;
  
@@@ -150,9 -146,12 +146,12 @@@
       * @param discoEvt Discovery event.
       * @param exchId Exchange id.
       */
-     public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, 
boolean reassign, DiscoveryEvent discoEvt,
-         GridDhtPartitionExchangeId exchId) {
-         super(cctx.kernalContext());
+     public GridDhtPartitionsExchangeFuture(
 -        GridCacheSharedContext<K, V> cctx,
++        GridCacheSharedContext cctx,
+         boolean reassign,
+         DiscoveryEvent discoEvt,
+         GridDhtPartitionExchangeId exchId
+     ) {
          dummy = true;
          forcePreload = false;
  
@@@ -174,9 -171,8 +171,8 @@@
       * @param discoEvt Discovery event.
       * @param exchId Exchange id.
       */
 -    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext<K, V> cctx, 
DiscoveryEvent discoEvt,
 +    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, 
DiscoveryEvent discoEvt,
          GridDhtPartitionExchangeId exchId) {
-         super(cctx.kernalContext());
          dummy = false;
          forcePreload = true;
  
@@@ -196,12 -190,8 +190,8 @@@
       * @param busyLock Busy lock.
       * @param exchId Exchange ID.
       */
 -    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext<K, V> cctx, 
ReadWriteLock busyLock,
 +    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, 
ReadWriteLock busyLock,
          GridDhtPartitionExchangeId exchId) {
-         super(cctx.kernalContext());
- 
-         syncNotify(true);
- 
          assert busyLock != null;
          assert exchId != null;
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index d27f516,9b738fc..2656d27
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@@ -365,11 -374,8 +365,11 @@@ public class GridNearAtomicCache<K, V> 
          ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
  
          if (F.isEmpty(keys))
-             return new GridFinishedFuture<>(ctx.kernalContext(), 
Collections.<K, V>emptyMap());
+             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
  
 +        if (keyCheck)
 +            validateCacheKeys(keys);
 +
          GridCacheProjectionImpl<K, V> prj = ctx.projectionPerCall();
  
          subjId = ctx.subjectIdPerCall(subjId, prj);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e59ea498/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --cc 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index ce8c995,c112547..e33b6a2
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@@ -242,9 -277,12 +242,9 @@@ public abstract class GridNearCacheAdap
          boolean skipVal
      ) {
          if (F.isEmpty(keys))
-             return new GridFinishedFuture<>(ctx.kernalContext(), 
Collections.<K, V>emptyMap());
+             return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
  
 -        if (keyCheck)
 -            validateCacheKeys(keys);
 -
 -        IgniteTxLocalEx<K, V> txx = (tx != null && tx.local()) ? 
(IgniteTxLocalEx<K, V>)tx : null;
 +        IgniteTxLocalEx txx = (tx != null && tx.local()) ? 
(IgniteTxLocalEx)tx : null;
  
          final IgniteCacheExpiryPolicy expiry = expiryPolicy(expiryPlc);
  

Reply via email to