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

[01/41] ignite git commit: 2.1 More simple tx cancel on node stop

Repository: ignite
Updated Branches:
  refs/heads/ignite-5578-1 [created] 5a38c46de


2.1 More simple tx cancel on node stop


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

Branch: refs/heads/ignite-5578-1
Commit: ab5267134966b466a2d4bfba15d0c7e3ebe91139
Parents: 31e9d3b
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 13:14:18 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 13:14:18 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java         |  2 ++
 .../cache/distributed/dht/GridDhtTxFinishFuture.java |  4 +++-
 .../distributed/near/GridNearTxFinishFuture.java     |  4 ++--
 .../cache/transactions/IgniteTxAdapter.java          |  4 ++--
 .../cache/transactions/IgniteTxLocalAdapter.java     | 15 +++++++++------
 5 files changed, 18 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ab526713/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 716482e..321e6dd 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
@@ -964,6 +964,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         // No new caches should be added after this point.
         exch.onKernalStop(cancel);
 
+        sharedCtx.mvcc().onStop();
+
         for (CacheGroupContext grp : cacheGrps.values())
             grp.onKernalStop();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab526713/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git 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
index d8180b4..5311ddc 100644
--- 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
@@ -225,7 +225,9 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCacheCompoundIdentity
 
             if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) {
                 try {
-                    this.tx.tmFinish(err == null);
+                    boolean nodeStop = err != null && X.hasCause(err, NodeStoppingException.class);
+
+                    this.tx.tmFinish(err == null, nodeStop);
                 }
                 catch (IgniteCheckedException finishErr) {
                     U.error(log, "Failed to finish tx: " + tx, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab526713/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 7f6f793..c45eb7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -327,7 +327,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
                         finishOnePhase(commit);
 
                     try {
-                        tx.tmFinish(commit);
+                        tx.tmFinish(commit, nodeStop);
                     }
                     catch (IgniteCheckedException e) {
                         U.error(log, "Failed to finish tx: " + tx, e);
@@ -338,7 +338,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCacheCompoundIdentit
                 }
 
                 if (super.onDone(tx0, err)) {
-                    if (error() instanceof IgniteTxHeuristicCheckedException) {
+                    if (error() instanceof IgniteTxHeuristicCheckedException && !nodeStop) {
                         AffinityTopologyVersion topVer = tx.topologyVersion();
 
                         for (IgniteTxEntry e : tx.writeMap().values()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab526713/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 51956ac..880d9b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -435,9 +435,9 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
                         break;
                     }
                 }
-            }
 
-            cctx.tm().uncommitTx(this);
+                cctx.tm().uncommitTx(this);
+            }
         }
         catch (Exception ex) {
             U.error(log, "Failed to do uncommit.", ex);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ab526713/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
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 49b67da..e7ebaae 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
@@ -898,17 +898,20 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
      * Commits transaction to transaction manager. Used for one-phase commit transactions only.
      *
      * @param commit If {@code true} commits transaction, otherwise rollbacks.
+     * @param nodeStop If {@code true} tx is cancelled on node stop.
      * @throws IgniteCheckedException If failed.
      */
-    public void tmFinish(boolean commit) throws IgniteCheckedException {
+    public void tmFinish(boolean commit, boolean nodeStop) throws IgniteCheckedException {
         assert onePhaseCommit();
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {
-            // Unlock all locks.
-            if (commit)
-                cctx.tm().commitTx(this);
-            else
-                cctx.tm().rollbackTx(this);
+            if (!nodeStop) {
+                // Unlock all locks.
+                if (commit)
+                    cctx.tm().commitTx(this);
+                else
+                    cctx.tm().rollbackTx(this);
+            }
 
             state(commit ? COMMITTED : ROLLED_BACK);
 


[08/41] ignite git commit: Fixed formatting.

Posted by sb...@apache.org.
Fixed formatting.


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

Branch: refs/heads/ignite-5578-1
Commit: 7adb11109bab5d83ed4f376b0cad42b026dd0a71
Parents: 984ab5b
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 14:49:13 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 14:49:13 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 52 +++++++++++++-------
 1 file changed, 34 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7adb1110/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 320ef06..f49dccf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -444,23 +444,28 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     if (stopping)
                         return;
 
-                    GridDhtPartitionExchangeId exchId = exchFut.exchangeId();assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" +
-                        topVer + ", exchId=" + exchId + ']';
+                    GridDhtPartitionExchangeId exchId = exchFut.exchangeId();
+
+                    assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" + topVer +
+                        ", exchId=" + exchId + ']';
 
                     if (exchId.isLeft() && exchFut.serverNodeDiscoveryEvent())
                         removeNode(exchId.nodeId());
     
                     ClusterNode oldest = discoCache.oldestAliveServerNodeWithCache();
 
-                    if (log.isDebugEnabled())
-                        log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
+                    if (log.isDebugEnabled()) {
+                        log.debug("Partition map beforeExchange [exchId=" + exchId +
+                            ", fullMap=" + fullMapString() + ']');
+                    }
 
                     long updateSeq = this.updateSeq.incrementAndGet();
 
                     cntrMap.clear();
 
-                    boolean grpStarted = exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom());// If this is the oldest node.
+                    boolean grpStarted = exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom());
 
+                    // If this is the oldest node.
                     if (oldest != null && (loc.equals(oldest) || grpStarted)) {
                         if (node2part == null) {
                             node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
@@ -470,18 +475,28 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                     exchId + ", fullMap=" + fullMapString() + ']');
                         }
                         else if (!node2part.valid()) {
-                            node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq, node2part, false);
-
-                            if (log.isDebugEnabled())
-                                log.debug("Created new full topology map on oldest node [exchId=" + exchId + ", fullMap=" +
-                                    node2part + ']');
+                            node2part = new GridDhtPartitionFullMap(oldest.id(),
+                                oldest.order(),
+                                updateSeq,
+                                node2part,
+                                false);
+
+                            if (log.isDebugEnabled()) {
+                                log.debug("Created new full topology map on oldest node [exchId=" + exchId +
+                                    ", fullMap=" + node2part + ']');
+                            }
                         }
                         else if (!node2part.nodeId().equals(loc.id())) {
-                            node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq, node2part, false);
-
-                            if (log.isDebugEnabled())
-                                log.debug("Copied old map into new map on oldest node (previous oldest node left) [exchId=" +
-                                    exchId + ", fullMap=" + fullMapString() + ']');
+                            node2part = new GridDhtPartitionFullMap(oldest.id(),
+                                oldest.order(),
+                                updateSeq,
+                                node2part,
+                                false);
+
+                            if (log.isDebugEnabled()) {
+                                log.debug("Copied old map into new map on oldest node (previous oldest node left) [" +
+                                    "exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
+                            }
                         }
                     }
 
@@ -499,9 +514,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                     consistencyCheck();
 
-                    if (log.isDebugEnabled())
-                        log.debug("Partition map after beforeExchange [exchId=" + exchId + ", fullMap=" +
-                            fullMapString() + ']');
+                    if (log.isDebugEnabled()) {
+                        log.debug("Partition map after beforeExchange [exchId=" + exchId +
+                            ", fullMap=" + fullMapString() + ']');
+                    }
                 }
                 finally {
                     lock.writeLock().unlock();


[04/41] ignite git commit: ignite-2.1 Added map with previous snapshots.

Posted by sb...@apache.org.
ignite-2.1 Added map with previous snapshots.


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

Branch: refs/heads/ignite-5578-1
Commit: 30922ed8ac518749a2559aff51fe183a890041bb
Parents: f589628
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Jul 7 18:03:18 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Jul 7 18:03:18 2017 +0700

----------------------------------------------------------------------
 .../pagemem/snapshot/SnapshotOperation.java        | 17 ++++++++++++++++-
 1 file changed, 16 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/30922ed8/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
index 863107a..fa18cd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.pagemem.snapshot;
 import java.io.File;
 import java.io.Serializable;
 import java.util.Collection;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -50,6 +51,9 @@ public class SnapshotOperation implements Serializable {
     /** Optional list of dependent snapshot IDs. */
     private final Set<Long> dependentSnapshotIds;
 
+    /** Optional map of previous snapshots grouped by caches. */
+    private final Map<Long, Set<String>> prevSnapshots;
+
     /**
      * @param type Type.
      * @param snapshotId Snapshot id.
@@ -58,6 +62,7 @@ public class SnapshotOperation implements Serializable {
      * @param msg Extra user message.
      * @param extraParam Additional parameter.
      * @param dependentSnapshotIds Optional list of dependent snapshot IDs.
+     * @param prevSnapshots Optional map of previous snapshots grouped by caches.
      */
     public SnapshotOperation(
         SnapshotOperationType type,
@@ -66,7 +71,8 @@ public class SnapshotOperation implements Serializable {
         Set<String> cacheNames,
         String msg,
         Object extraParam,
-        Set<Long> dependentSnapshotIds
+        Set<Long> dependentSnapshotIds,
+        Map<Long, Set<String>> prevSnapshots
     ) {
         this.type = type;
         this.snapshotId = snapshotId;
@@ -75,6 +81,7 @@ public class SnapshotOperation implements Serializable {
         this.msg = msg;
         this.extraParam = extraParam;
         this.dependentSnapshotIds = dependentSnapshotIds;
+        this.prevSnapshots = prevSnapshots;
     }
 
     /**
@@ -131,6 +138,13 @@ public class SnapshotOperation implements Serializable {
     }
 
     /**
+     * @return Cache names grouped by previous snapshot IDs.
+     */
+    public Map<Long, Set<String>> previousSnapshots() {
+        return prevSnapshots;
+    }
+
+    /**
      * @param op Op.
      */
     public static Collection<File> getOptionalPathsParameter(SnapshotOperation op) {
@@ -215,6 +229,7 @@ public class SnapshotOperation implements Serializable {
             ", msg='" + msg + '\'' +
             ", extraParam=" + extraParam +
             ", dependentSnapshotIds=" + dependentSnapshotIds +
+            ", prevSnapshots=" + prevSnapshots +
             '}';
     }
 }


[21/41] ignite git commit: IGNITE-5701 - Some nodes have partitionUpdateCounter equal to 0 after rebalancing

Posted by sb...@apache.org.
IGNITE-5701 - Some nodes have partitionUpdateCounter equal to 0 after rebalancing


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

Branch: refs/heads/ignite-5578-1
Commit: 993f7fbe1d49a524e2dee626aef72e16fd5d3cda
Parents: 517a23d
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Fri Jul 7 18:55:27 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Jul 7 18:55:41 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  2 +-
 .../distributed/dht/GridDhtCacheEntry.java      |  6 --
 .../distributed/dht/GridDhtLocalPartition.java  | 45 ------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 16 ++++-
 .../dht/preloader/GridDhtPartitionDemander.java |  5 +-
 .../dht/preloader/GridDhtPartitionSupplier.java |  2 +-
 .../GridDhtPartitionSupplyMessage.java          | 20 +++---
 .../GridCacheDatabaseSharedManager.java         |  2 +-
 .../IgnitePdsCacheRebalancingAbstractTest.java  | 74 ++++++++++++++++++++
 .../wal/IgniteWalHistoryReservationsTest.java   | 29 ++++++--
 10 files changed, 131 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 085712a..35b0577 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -626,7 +626,7 @@ public final class IgniteSystemProperties {
     /**
      * WAL rebalance threshold.
      */
-     public static final String IGNITE_PDS_WAL_REBALANCE_THRESHOLD = "IGNITE_PDS_WAL_REBALANCE_THRESHOLD";
+    public static final String IGNITE_PDS_WAL_REBALANCE_THRESHOLD = "IGNITE_PDS_WAL_REBALANCE_THRESHOLD";
 
     /** Ignite page memory concurrency level. */
     public static final String IGNITE_OFFHEAP_LOCK_CONCURRENCY_LEVEL = "IGNITE_OFFHEAP_LOCK_CONCURRENCY_LEVEL";

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git 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
index 57dd622..77cc642 100644
--- 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
@@ -123,12 +123,6 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
     }
 
     /** {@inheritDoc} */
-    @Override protected void onUpdateFinished(long cntr) {
-        if (cctx.shared().database().persistenceEnabled())
-            locPart.onUpdateReceived(cntr);
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean isDht() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 8e42351..725822d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -157,13 +157,6 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
     @GridToStringExclude
     private final CacheDataStore store;
 
-    /** Partition updates. */
-    @GridToStringExclude
-    private final ConcurrentNavigableMap<Long, Boolean> updates = new ConcurrentSkipListMap<>();
-
-    /** Last applied update. */
-    private final AtomicLong lastApplied = new AtomicLong(0);
-
     /** Set if failed to move partition to RENTING state due to reservations, to be checked when
      * reservation is released. */
     private volatile boolean shouldBeRenting;
@@ -349,44 +342,6 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
     }
 
     /**
-     * @return Last applied update.
-     */
-    public long lastAppliedUpdate() {
-        return lastApplied.get();
-    }
-
-    /**
-     * @param cntr Received counter.
-     */
-    public void onUpdateReceived(long cntr) {
-        boolean changed = updates.putIfAbsent(cntr, true) == null;
-
-        if (!changed)
-            return;
-
-        while (true) {
-            Map.Entry<Long, Boolean> entry = updates.firstEntry();
-
-            if (entry == null)
-                return;
-
-            long first = entry.getKey();
-
-            long cntr0 = lastApplied.get();
-
-            if (first <= cntr0)
-                updates.remove(first);
-            else if (first == cntr0 + 1)
-                if (lastApplied.compareAndSet(cntr0, first))
-                    updates.remove(first);
-                else
-                    break;
-            else
-                break;
-        }
-    }
-
-    /**
      * @return If partition is moving or owning or renting.
      */
     public boolean valid() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index cf0dd5f..2f54810 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -673,6 +673,11 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         if (loc == null || loc.state() == EVICTED) {
             locParts.set(p, loc = new GridDhtLocalPartition(ctx, grp, p));
 
+            T2<Long, Long> cntr = cntrMap.get(p);
+
+            if (cntr != null)
+                loc.updateCounter(cntr.get2());
+
             if (ctx.pageStore() != null) {
                 try {
                     ctx.pageStore().onPartitionCreated(grp.groupId(), p);
@@ -1334,11 +1339,12 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 if (cntr != null && cntr.get2() > part.updateCounter())
                     part.updateCounter(cntr.get2());
+                else if (part.updateCounter() > 0)
+                    this.cntrMap.put(part.id(), new T2<>(part.initialUpdateCounter(), part.updateCounter()));
             }
         }
         finally {
             lock.writeLock().unlock();
-
         }
     }
 
@@ -1715,6 +1721,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         result.add(ctx.localNodeId());
                     }
 
+                    U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
+                        "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
+                        ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]");
+
                 }
             }
 
@@ -1731,6 +1741,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         result.add(e.getKey());
                     }
                 }
+
+                U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
+                    "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
+                    ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]");
             }
 
             if (updateSeq)

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index e7e95b2..4f34aba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -640,7 +640,7 @@ public class GridDhtPartitionDemander {
 
                     assert part != null;
 
-                    boolean last = supply.last().contains(p);
+                    boolean last = supply.last().containsKey(p);
 
                     if (part.state() == MOVING) {
                         boolean reserved = part.reserve();
@@ -680,6 +680,9 @@ public class GridDhtPartitionDemander {
                             // If message was last for this partition,
                             // then we take ownership.
                             if (last) {
+                                if (supply.isClean(p))
+                                    part.updateCounter(supply.last().get(p));
+
                                 top.own(part);
 
                                 fut.partitionDone(id, p);

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
index 1cc6c28..3ead982 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
@@ -414,7 +414,7 @@ class GridDhtPartitionSupplier {
                     }
 
                     // Mark as last supply message.
-                    s.last(part);
+                    s.last(part, loc.updateCounter());
 
                     phase = SupplyContextPhase.NEW;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
index ef14a90..90d11f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
@@ -56,8 +56,8 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
     private AffinityTopologyVersion topVer;
 
     /** Partitions that have been fully sent. */
-    @GridDirectCollection(int.class)
-    private Collection<Integer> last;
+    @GridDirectMap(keyType = int.class, valueType = long.class)
+    private Map<Integer, Long> last;
 
     /** Partitions which were not found. */
     @GridToStringInclude
@@ -128,19 +128,19 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
     /**
      * @return Flag to indicate last message for partition.
      */
-    Collection<Integer> last() {
-        return last == null ? Collections.<Integer>emptySet() : last;
+    Map<Integer, Long> last() {
+        return last == null ? Collections.<Integer, Long>emptyMap() : last;
     }
 
     /**
      * @param p Partition which was fully sent.
      */
-    void last(int p) {
+    void last(int p, long cntr) {
         if (last == null)
-            last = new HashSet<>();
+            last = new HashMap<>();
 
-        if (last.add(p)) {
-            msgSize += 4;
+        if (last.put(p, cntr) == null) {
+            msgSize += 12;
 
             // If partition is empty, we need to add it.
             if (!infos().containsKey(p)) {
@@ -304,7 +304,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeCollection("last", last, MessageCollectionItemType.INT))
+                if (!writer.writeMap("last", last, MessageCollectionItemType.INT, MessageCollectionItemType.LONG))
                     return false;
 
                 writer.incrementState();
@@ -382,7 +382,7 @@ public class GridDhtPartitionSupplyMessage extends GridCacheGroupIdMessage imple
                 reader.incrementState();
 
             case 7:
-                last = reader.readCollection("last", MessageCollectionItemType.INT);
+                last = reader.readMap("last", MessageCollectionItemType.INT, MessageCollectionItemType.LONG, false);
 
                 if (!reader.isLastRead())
                     return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 4af4daf..d64677e 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
@@ -2126,7 +2126,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     CacheState state = new CacheState(locParts.size());
 
                     for (GridDhtLocalPartition part : grp.topology().currentLocalPartitions())
-                        state.addPartitionState(part.id(), part.dataStore().fullSize(), part.lastAppliedUpdate());
+                        state.addPartitionState(part.id(), part.dataStore().fullSize(), part.updateCounter());
 
                     cpRec.addCacheGroupState(grp.groupId(), state);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
index cbc2623..588b3ac 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheRebalancingAbstractTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.persistence;
 
 import java.io.Serializable;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.Callable;
@@ -29,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.QueryEntity;
@@ -41,11 +43,13 @@ import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 
@@ -489,6 +493,76 @@ public abstract class IgnitePdsCacheRebalancingAbstractTest extends GridCommonAb
     }
 
     /**
+     * @throws Exception If failed
+     */
+    public void testPartitionCounterConsistencyOnUnstableTopology() throws Exception {
+        final Ignite ig = startGrids(4);
+
+        ig.active(true);
+
+        int k = 0;
+
+        try (IgniteDataStreamer ds = ig.dataStreamer(cacheName)) {
+            ds.allowOverwrite(true);
+
+            for (int k0 = k; k < k0 + 10_000; k++)
+                ds.addData(k, k);
+        }
+
+        for (int t = 0; t < 10; t++) {
+            IgniteInternalFuture fut = GridTestUtils.runAsync(new Runnable() {
+                @Override public void run() {
+                    try {
+                        stopGrid(3);
+
+                        IgniteEx ig0 = startGrid(3);
+
+                        awaitPartitionMapExchange();
+
+                        ig0.cache(cacheName).rebalance().get();
+                    }
+                    catch (Exception e) {
+                        throw new RuntimeException(e);
+                    }
+                }
+            });
+
+            try (IgniteDataStreamer ds = ig.dataStreamer(cacheName)) {
+                ds.allowOverwrite(true);
+
+                while (!fut.isDone()) {
+                    ds.addData(k, k);
+
+                    k++;
+
+                    U.sleep(1);
+                }
+            }
+
+            fut.get();
+
+            Map<Integer, Long> cntrs = new HashMap<>();
+
+            for (int g = 0; g < 4; g++) {
+                IgniteEx ig0 = grid(g);
+
+                for (GridDhtLocalPartition part : ig0.cachex(cacheName).context().topology().currentLocalPartitions()) {
+                    if (cntrs.containsKey(part.id()))
+                        assertEquals(String.valueOf(part.id()), (long) cntrs.get(part.id()), part.updateCounter());
+                    else
+                        cntrs.put(part.id(), part.updateCounter());
+                }
+
+                for (int k0 = 0; k0 < k; k0++) {
+                    assertEquals(String.valueOf(k0), k0, ig0.cache(cacheName).get(k0));
+                }
+            }
+
+            assertEquals(ig.affinity(cacheName).partitions(), cntrs.size());
+        }
+    }
+
+    /**
      *
      */
     private static class TestValue implements Serializable {

http://git-wip-us.apache.org/repos/asf/ignite/blob/993f7fbe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
index 48d8c21..4bea63f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalHistoryReservationsTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
@@ -201,7 +202,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
 
         int entryCnt = 10_000;
 
-        Ignite ig0 = startGrids(2);
+        IgniteEx ig0 = (IgniteEx) startGrids(2);
 
         ig0.active(true);
 
@@ -219,7 +220,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
 
         forceCheckpoint();
 
-        Ignite ig1 = startGrid(1);
+        IgniteEx ig1 = startGrid(1);
 
         IgniteCache<Integer, Integer> cache1 = ig1.cache("cache1");
 
@@ -236,6 +237,16 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
                 assertEquals("k=" + k, k, cache1.get(k));
             }
         }
+
+        cache.rebalance().get();
+
+        for (int p = 0; p < ig1.affinity("cache1").partitions(); p++) {
+            GridDhtLocalPartition p0 = ig0.context().cache().cache("cache1").context().topology().localPartition(p);
+            GridDhtLocalPartition p1 = ig1.context().cache().cache("cache1").context().topology().localPartition(p);
+
+            assertTrue(p0.updateCounter() > 0);
+            assertEquals(p0.updateCounter(), p1.updateCounter());
+        }
     }
 
     /**
@@ -244,7 +255,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
     public void testNodeIsClearedIfHistoryIsUnavailable() throws Exception {
         int entryCnt = 10_000;
 
-        Ignite ig0 = startGrids(2);
+        IgniteEx ig0 = (IgniteEx) startGrids(2);
 
         ig0.active(true);
 
@@ -269,7 +280,7 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
                 assertEquals("k=" + k, k, cache.get(k));
         }
 
-        Ignite ig1 = startGrid(1);
+        IgniteEx ig1 = startGrid(1);
 
         IgniteCache<Integer, Integer> cache1 = ig1.cache("cache1");
 
@@ -286,6 +297,16 @@ public class IgniteWalHistoryReservationsTest extends GridCommonAbstractTest {
                 assertEquals("k=" + k, k, cache1.get(k));
             }
         }
+
+        cache.rebalance().get();
+
+        for (int p = 0; p < ig1.affinity("cache1").partitions(); p++) {
+            GridDhtLocalPartition p0 = ig0.context().cache().cache("cache1").context().topology().localPartition(p);
+            GridDhtLocalPartition p1 = ig1.context().cache().cache("cache1").context().topology().localPartition(p);
+
+            assertTrue(p0.updateCounter() > 0);
+            assertEquals(p0.updateCounter(), p1.updateCounter());
+        }
     }
 
     /**


[03/41] ignite git commit: Rename ActionData -> CacheActionData.

Posted by sb...@apache.org.
Rename ActionData -> CacheActionData.


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

Branch: refs/heads/ignite-5578-1
Commit: f589628f4846f14efc8e702b61856cc90a3d0dc7
Parents: 85d8c65
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 13:52:38 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 13:52:38 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java       | 10 +++---
 .../processors/cache/ClusterCachesInfo.java     |  2 +-
 .../processors/cache/ExchangeActions.java       | 34 ++++++++++----------
 .../processors/cache/GridCacheProcessor.java    |  2 +-
 .../GridDhtPartitionsExchangeFuture.java        |  2 +-
 5 files changed, 25 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f589628f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
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 8d08c3f..548d795 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
@@ -701,7 +701,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             }
         });
 
-        for (ExchangeActions.ActionData action : exchActions.cacheStartRequests()) {
+        for (ExchangeActions.CacheActionData action : exchActions.cacheStartRequests()) {
             DynamicCacheDescriptor cacheDesc = action.descriptor();
 
             DynamicCacheChangeRequest req = action.request();
@@ -768,7 +768,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
         Set<Integer> gprs = new HashSet<>();
 
-        for (ExchangeActions.ActionData action : exchActions.cacheStartRequests()) {
+        for (ExchangeActions.CacheActionData action : exchActions.cacheStartRequests()) {
             Integer grpId = action.descriptor().groupId();
 
             if (gprs.add(grpId)) {
@@ -786,7 +786,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
             }
         }
 
-        for (ExchangeActions.ActionData action : exchActions.cacheStopRequests())
+        for (ExchangeActions.CacheActionData action : exchActions.cacheStopRequests())
             cctx.cache().blockGateway(action.request().cacheName(), true, action.request().restart());
 
         for (ExchangeActions.CacheGroupActionData action : exchActions.cacheGroupsToStop()) {
@@ -2308,10 +2308,10 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
                 assert old == null : old;
             }
 
-            for (ExchangeActions.ActionData req : exchActions.cacheStopRequests())
+            for (ExchangeActions.CacheActionData req : exchActions.cacheStopRequests())
                 registeredCaches.remove(req.descriptor().cacheId());
 
-            for (ExchangeActions.ActionData req : exchActions.cacheStartRequests())
+            for (ExchangeActions.CacheActionData req : exchActions.cacheStartRequests())
                 registeredCaches.put(req.descriptor().cacheId(), req.descriptor());
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f589628f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
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 5aca8c9..738e4ac 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
@@ -624,7 +624,7 @@ class ClusterCachesInfo {
                         // If all caches in group will be destroyed it is not necessary to destroy single cache
                         // because group will be stopped anyway.
                         if (req.destroy()) {
-                            for (ExchangeActions.ActionData action : exchangeActions.cacheStopRequests()) {
+                            for (ExchangeActions.CacheActionData action : exchangeActions.cacheStopRequests()) {
                                 if (action.descriptor().groupId() == grpDesc.groupId())
                                     action.request().destroy(false);
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f589628f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
index e9ece5a..1cc6438 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java
@@ -41,13 +41,13 @@ public class ExchangeActions {
     private List<CacheGroupActionData> cacheGrpsToStop;
 
     /** */
-    private Map<String, ActionData> cachesToStart;
+    private Map<String, CacheActionData> cachesToStart;
 
     /** */
-    private Map<String, ActionData> cachesToStop;
+    private Map<String, CacheActionData> cachesToStop;
 
     /** */
-    private Map<String, ActionData> cachesToResetLostParts;
+    private Map<String, CacheActionData> cachesToResetLostParts;
 
     /** */
     private StateChangeRequest stateChangeReq;
@@ -60,7 +60,7 @@ public class ExchangeActions {
         Boolean destroy = null;
 
         // Check that caches associated with that group will be all stopped only or all destroyed.
-        for (ExchangeActions.ActionData action : cacheStopRequests()) {
+        for (CacheActionData action : cacheStopRequests()) {
             if (action.descriptor().groupId() == grpId) {
                 if (destroy == null)
                     destroy = action.request().destroy();
@@ -89,15 +89,15 @@ public class ExchangeActions {
     /**
      * @return New caches start requests.
      */
-    public Collection<ActionData> cacheStartRequests() {
-        return cachesToStart != null ? cachesToStart.values() : Collections.<ActionData>emptyList();
+    public Collection<CacheActionData> cacheStartRequests() {
+        return cachesToStart != null ? cachesToStart.values() : Collections.<CacheActionData>emptyList();
     }
 
     /**
      * @return Stop cache requests.
      */
-    Collection<ActionData> cacheStopRequests() {
-        return cachesToStop != null ? cachesToStop.values() : Collections.<ActionData>emptyList();
+    Collection<CacheActionData> cacheStopRequests() {
+        return cachesToStop != null ? cachesToStop.values() : Collections.<CacheActionData>emptyList();
     }
 
     /**
@@ -114,7 +114,7 @@ public class ExchangeActions {
      */
     public boolean systemCachesStarting() {
         if (cachesToStart != null) {
-            for (ActionData data : cachesToStart.values()) {
+            for (CacheActionData data : cachesToStart.values()) {
                 if (CU.isSystemCache(data.request().cacheName()))
                     return true;
             }
@@ -127,9 +127,9 @@ public class ExchangeActions {
      * @param map Actions map.
      * @param ctx Context.
      */
-    private void completeRequestFutures(Map<String, ActionData> map, GridCacheSharedContext ctx) {
+    private void completeRequestFutures(Map<String, CacheActionData> map, GridCacheSharedContext ctx) {
         if (map != null) {
-            for (ActionData req : map.values())
+            for (CacheActionData req : map.values())
                 ctx.cache().completeCacheStartFuture(req.req, true, null);
         }
     }
@@ -159,7 +159,7 @@ public class ExchangeActions {
      */
     public boolean cacheStopped(int cacheId) {
         if (cachesToStop != null) {
-            for (ActionData cache : cachesToStop.values()) {
+            for (CacheActionData cache : cachesToStop.values()) {
                 if (cache.desc.cacheId() == cacheId)
                     return true;
             }
@@ -174,7 +174,7 @@ public class ExchangeActions {
      */
     public boolean cacheStarted(int cacheId) {
         if (cachesToStart != null) {
-            for (ActionData cache : cachesToStart.values()) {
+            for (CacheActionData cache : cachesToStart.values()) {
                 if (cache.desc.cacheId() == cacheId)
                     return true;
             }
@@ -217,7 +217,7 @@ public class ExchangeActions {
      * @param desc Cache descriptor.
      * @return Actions map.
      */
-    private Map<String, ActionData> add(Map<String, ActionData> map,
+    private Map<String, CacheActionData> add(Map<String, CacheActionData> map,
         DynamicCacheChangeRequest req,
         DynamicCacheDescriptor desc) {
         assert req != null;
@@ -226,7 +226,7 @@ public class ExchangeActions {
         if (map == null)
             map = new HashMap<>();
 
-        ActionData old = map.put(req.cacheName(), new ActionData(req, desc));
+        CacheActionData old = map.put(req.cacheName(), new CacheActionData(req, desc));
 
         assert old == null : old;
 
@@ -347,7 +347,7 @@ public class ExchangeActions {
     /**
      *
      */
-    public static class ActionData {
+    public static class CacheActionData {
         /** */
         private final DynamicCacheChangeRequest req;
 
@@ -358,7 +358,7 @@ public class ExchangeActions {
          * @param req Request.
          * @param desc Cache descriptor.
          */
-        ActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
+        CacheActionData(DynamicCacheChangeRequest req, DynamicCacheDescriptor desc) {
             assert req != null;
             assert desc != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f589628f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 321e6dd..0a69d72 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
@@ -2126,7 +2126,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 }
             }
 
-            for (ExchangeActions.ActionData action : exchActions.cacheStopRequests()) {
+            for (ExchangeActions.CacheActionData action : exchActions.cacheStopRequests()) {
                 stopGateway(action.request());
 
                 sharedCtx.database().checkpointReadLock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f589628f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 2151101..97fcb12 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
@@ -767,7 +767,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 if (cctx.database().persistenceEnabled() && !cctx.kernalContext().clientNode()) {
                     List<DynamicCacheDescriptor> startDescs = new ArrayList<>();
 
-                    for (ExchangeActions.ActionData startReq : exchActions.cacheStartRequests())
+                    for (ExchangeActions.CacheActionData startReq : exchActions.cacheStartRequests())
                         startDescs.add(startReq.descriptor());
 
                     cctx.database().readCheckpointAndRestoreMemory(startDescs);


[30/41] ignite git commit: Fixed NPE in setOwners.

Posted by sb...@apache.org.
Fixed NPE in setOwners.


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

Branch: refs/heads/ignite-5578-1
Commit: 0b9527e851fd7c1d63bfc427721ca8af80d54b0c
Parents: a535444
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 12:02:03 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 12:02:03 2017 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtPartitionTopologyImpl.java    | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b9527e8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 2f54810..7c62cca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1722,7 +1722,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     }
 
                     U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
-                        "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
+                        "[nodeId=" + ctx.localNodeId() + ", cacheOrGroupName=" + grp.cacheOrGroupName() +
                         ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]");
 
                 }
@@ -1743,8 +1743,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 }
 
                 U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
-                    "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
-                    ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]");
+                    "[nodeId=" + ctx.localNodeId() + ", cacheOrGroupName=" + grp.cacheOrGroupName() +
+                    ", partId=" + p + ", haveHistory=" + haveHistory + "]");
             }
 
             if (updateSeq)


[28/41] ignite git commit: ignite-5446 Alway use late affinity assignment mode

Posted by sb...@apache.org.
ignite-5446 Alway use late affinity assignment mode


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

Branch: refs/heads/ignite-5578-1
Commit: a53544410dd15a3a5112d6de88648db21bd3fcf3
Parents: 17904cb
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 11:38:04 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 11:38:04 2017 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  15 +-
 .../affinity/AffinityHistoryCleanupTest.java    | 182 -------------------
 ...idAbstractCacheInterceptorRebalanceTest.java |   2 -
 .../cache/GridCacheDeploymentSelfTest.java      |   7 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  22 +--
 ...ffinityAssignmentNodeJoinValidationTest.java |  46 +----
 .../CacheLateAffinityAssignmentTest.java        |   2 -
 ...CacheLoadingConcurrentGridStartSelfTest.java |   2 -
 ...idCachePartitionedPreloadEventsSelfTest.java | 143 ---------------
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |  35 ----
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |  34 ----
 .../db/IgnitePdsWholeClusterRestartTest.java    |   1 -
 .../GridActivationPartitionedCacheSuit.java     |   2 -
 ...ContinuousQueryFailoverAbstractSelfTest.java |   2 -
 .../processors/igfs/IgfsStreamsSelfTest.java    |   1 -
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 -
 .../testsuites/IgniteCacheTestSuite2.java       |   2 -
 17 files changed, 17 insertions(+), 487 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ed05fa4..cafa675 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -191,6 +191,7 @@ public class IgniteConfiguration {
     public static final boolean DFLT_CACHE_SANITY_CHECK_ENABLED = true;
 
     /** Default value for late affinity assignment flag. */
+    @Deprecated
     public static final boolean DFLT_LATE_AFF_ASSIGNMENT = true;
 
     /** Default value for active on start flag. */
@@ -452,9 +453,6 @@ public class IgniteConfiguration {
     /** Custom executor configurations. */
     private ExecutorConfiguration[] execCfgs;
 
-    /** */
-    private boolean lateAffAssignment = DFLT_LATE_AFF_ASSIGNMENT;
-
     /** Page memory configuration. */
     private MemoryConfiguration memCfg;
 
@@ -530,7 +528,6 @@ public class IgniteConfiguration {
         igniteWorkDir = cfg.getWorkDirectory();
         inclEvtTypes = cfg.getIncludeEventTypes();
         includeProps = cfg.getIncludeProperties();
-        lateAffAssignment = cfg.isLateAffinityAssignment();
         lifecycleBeans = cfg.getLifecycleBeans();
         locHost = cfg.getLocalHost();
         log = cfg.getGridLogger();
@@ -2721,14 +2718,14 @@ public class IgniteConfiguration {
      * from assignment calculated by {@link AffinityFunction#assignPartitions}.
      * <p>
      * This property should have the same value for all nodes in cluster.
-     * <p>
-     * If not provided, default value is {@link #DFLT_LATE_AFF_ASSIGNMENT}.
      *
      * @return Late affinity assignment flag.
      * @see AffinityFunction
+     * @deprecated Starting from Ignite 2.1 late affinity assignment is always enabled.
      */
+    @Deprecated
     public boolean isLateAffinityAssignment() {
-        return lateAffAssignment;
+        return true;
     }
 
     /**
@@ -2736,10 +2733,10 @@ public class IgniteConfiguration {
      *
      * @param lateAffAssignment Late affinity assignment flag.
      * @return {@code this} for chaining.
+     * @deprecated Starting from Ignite 2.1 late affinity assignment is always enabled.
      */
+    @Deprecated
     public IgniteConfiguration setLateAffinityAssignment(boolean lateAffAssignment) {
-        this.lateAffAssignment = lateAffAssignment;
-
         return this;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
index 87c2050..605cc5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
@@ -47,9 +47,6 @@ public class AffinityHistoryCleanupTest extends GridCommonAbstractTest {
     /** */
     private boolean client;
 
-    /** */
-    private boolean lateAffAssignment;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
@@ -71,8 +68,6 @@ public class AffinityHistoryCleanupTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(client);
 
-        cfg.setLateAffinityAssignment(lateAffAssignment);
-
         return cfg;
     }
 
@@ -96,183 +91,6 @@ public class AffinityHistoryCleanupTest extends GridCommonAbstractTest {
 
             checkHistory(ignite, F.asList(topVer(1, 0)), 1);
 
-            for (int i = 0; i < 3; i++) {
-                startGrid(1);
-
-                stopGrid(1);
-            }
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0)),
-                5);
-
-            client = true;
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0),
-                topVer(8, 0),
-                topVer(9, 0)),
-                5);
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0),
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0)),
-                5);
-
-            startGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0),
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0)),
-                5);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0),
-                topVer(13, 0)),
-                0);
-
-            client = false;
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0),
-                topVer(13, 0),
-                topVer(14, 0),
-                topVer(15, 0)),
-                2);
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0),
-                topVer(13, 0),
-                topVer(14, 0),
-                topVer(15, 0),
-                topVer(16, 0),
-                topVer(17, 0)),
-                4);
-
-            startGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(13, 0),
-                topVer(14, 0),
-                topVer(15, 0),
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0)),
-                5);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(14, 0),
-                topVer(15, 0),
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0),
-                topVer(19, 0)),
-                6);
-
-            startGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0),
-                topVer(19, 0),
-                topVer(20, 0)),
-                5);
-
-            client = true;
-
-            startGrid(2);
-
-            stopGrid(2);
-
-            checkHistory(ignite, F.asList(
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0),
-                topVer(19, 0),
-                topVer(20, 0),
-                topVer(21, 0),
-                topVer(22, 0)),
-                5);
-        }
-        finally {
-            if (histProp != null)
-                System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, histProp);
-            else
-                System.clearProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityHistoryCleanupLateAffinityAssignment() throws Exception {
-        lateAffAssignment = true;
-
-        String histProp = System.getProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE);
-
-        try {
-            System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, "5");
-
-            Ignite ignite = startGrid(0);
-
-            checkHistory(ignite, F.asList(topVer(1, 0)), 1);
-
             startGrid(1);
 
             checkHistory(ignite, F.asList(

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index adfe085..99cf1f1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@ -75,8 +75,6 @@ public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridComm
     @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception {
         final IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         final CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>(CACHE_NAME);
 
         assertNotNull(interceptor);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
index c88d0cc..ff3ab36 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
@@ -92,8 +92,6 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
 
         cfg.setConnectorConfiguration(null);
 
-        cfg.setLateAffinityAssignment(false);
-
         return cfg;
     }
 
@@ -510,7 +508,10 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
      * @return Key with described properties.
      * @throws IllegalStateException if such a key could not be found after 10000 iterations.
      */
-    private int getNextKey(int start, Ignite g, ClusterNode primary, ClusterNode backup, ClusterNode near) {
+    private int getNextKey(int start, Ignite g, ClusterNode primary, ClusterNode backup, ClusterNode near)
+        throws Exception {
+        awaitPartitionMapExchange();
+
         info("Primary: " + primary);
         info("Backup: " + backup);
         info("Near: " + near);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
index 1a88d80..ba77c70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
@@ -22,7 +22,6 @@ import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.transactions.Transaction;
 
@@ -39,26 +38,13 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
         return CacheAtomicityMode.TRANSACTIONAL;
     }
 
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setLateAffinityAssignment(false);
-
-        if (!igniteInstanceName.endsWith("0"))
-            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); // Allows to check GridDhtLockRequest fail.
-
-        return cfg;
-    }
-
     /**
      * Sends put with optimistic lock and handles fail.
      */
-    protected void failOptimistic() {
+    private void failOptimistic() {
         IgniteCache<Object, Object> cache = jcache(0);
 
         try (Transaction tx = grid(0).transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
-
             cache.put(new TestKey(String.valueOf(++key)), "");
 
             tx.commit();
@@ -75,12 +61,10 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
     /**
      * Sends put with pessimistic lock and handles fail.
      */
-    protected void failPessimictic() {
+    private void failPessimictic() {
         IgniteCache<Object, Object> cache = jcache(0);
 
-        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC,
-            REPEATABLE_READ)) {
-
+        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
             cache.put(new TestKey(String.valueOf(++key)), "");
 
             assert false : "p2p marshalling failed, but error response was not sent";

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
index 11ac063..48b33b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
@@ -19,8 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -77,7 +75,8 @@ public class CacheLateAffinityAssignmentNodeJoinValidationTest extends GridCommo
      * @param firstEnabled Flag value for first started node.
      * @throws Exception If failed.
      */
-    public void checkNodeJoinValidation(boolean firstEnabled) throws Exception {
+    private void checkNodeJoinValidation(boolean firstEnabled) throws Exception {
+        // LateAffinity should be always enabled, setLateAffinityAssignment should be ignored.
         lateAff = firstEnabled;
 
         Ignite ignite = startGrid(0);
@@ -86,49 +85,12 @@ public class CacheLateAffinityAssignmentNodeJoinValidationTest extends GridCommo
 
         lateAff = !firstEnabled;
 
-        try {
-            startGrid(1);
-
-            fail();
-        }
-        catch (Exception e) {
-            checkError(e);
-        }
-
-        client = true;
-
-        try {
-            startGrid(1);
-
-            fail();
-        }
-        catch (Exception e) {
-            checkError(e);
-        }
-
-        assertEquals(1, ignite.cluster().nodes().size());
-
-        lateAff = firstEnabled;
-
-        client = false;
-
         startGrid(1);
 
         client = true;
 
-        Ignite client = startGrid(2);
-
-        assertTrue(client.configuration().isClientMode());
-    }
-
-    /**
-     * @param e Error.
-     */
-    private void checkError(Exception e) {
-        IgniteSpiException err = X.cause(e, IgniteSpiException.class);
+        startGrid(2);
 
-        assertNotNull(err);
-        assertTrue(err.getMessage().contains("Local node's cache affinity assignment mode differs " +
-            "from the same property on remote node"));
+        assertEquals(3, ignite.cluster().nodes().size());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
index 46520ca..6174209 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
@@ -142,8 +142,6 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         TestRecordingCommunicationSpi commSpi;
 
         if (spiC != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
index 4f1b090..68e88ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
@@ -79,8 +79,6 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
 
         CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java
deleted file mode 100644
index bc62a72..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java
+++ /dev/null
@@ -1,143 +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.distributed.dht;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.AffinityFunctionContext;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.events.Event;
-import org.apache.ignite.internal.processors.cache.distributed.GridCachePreloadEventsAbstractSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysFuture;
-import org.apache.ignite.internal.util.typedef.F;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
-
-/**
- *
- */
-public class GridCachePartitionedPreloadEventsSelfTest extends GridCachePreloadEventsAbstractSelfTest {
-    /** */
-    private boolean replicatedAffinity = true;
-
-    /** */
-    private long rebalanceDelay;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        // 'testForcePreload' is not valid with late assignment.
-        cfg.setLateAffinityAssignment(false);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration() {
-        CacheConfiguration cacheCfg = super.cacheConfiguration();
-
-        if (replicatedAffinity)
-            // replicate entries to all nodes
-            cacheCfg.setAffinity(notSerializableProxy(new AffinityFunction() {
-                /** {@inheritDoc} */
-                @Override public void reset() {
-                }
-
-                /** {@inheritDoc} */
-                @Override public int partitions() {
-                    return 1;
-                }
-
-                /** {@inheritDoc} */
-                @Override public int partition(Object key) {
-                    return 0;
-                }
-
-                /** {@inheritDoc} */
-                @Override public List<List<ClusterNode>> assignPartitions(AffinityFunctionContext affCtx) {
-                    List<ClusterNode> nodes = new ArrayList<>(affCtx.currentTopologySnapshot());
-
-                    return Collections.singletonList(nodes);
-                }
-
-                /** {@inheritDoc} */
-                @Override public void removeNode(UUID nodeId) {
-                }
-            }, AffinityFunction.class));
-
-        cacheCfg.setRebalanceDelay(rebalanceDelay);
-
-        return cacheCfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode getCacheMode() {
-        return PARTITIONED;
-    }
-
-    /**
-     * Test events fired from
-     * {@link GridDhtForceKeysFuture}
-     *
-     * @throws Exception if failed.
-     */
-    public void testForcePreload() throws Exception {
-        replicatedAffinity = false;
-        rebalanceDelay = -1;
-
-        Ignite g1 = startGrid("g1");
-
-        Collection<Integer> keys = new HashSet<>();
-
-        IgniteCache<Integer, String> cache = g1.cache(DEFAULT_CACHE_NAME);
-
-        for (int i = 0; i < 100; i++) {
-            keys.add(i);
-            cache.put(i, "val");
-        }
-
-        Ignite g2 = startGrid("g2");
-
-        Map<ClusterNode, Collection<Object>> keysMap = g1.affinity(DEFAULT_CACHE_NAME).mapKeysToNodes(keys);
-        Collection<Object> g2Keys = keysMap.get(g2.cluster().localNode());
-
-        assertNotNull(g2Keys);
-        assertFalse("There are no keys assigned to g2", g2Keys.isEmpty());
-
-        for (Object key : g2Keys)
-            // Need to force keys loading.
-            assertEquals("val", g2.cache(DEFAULT_CACHE_NAME).getAndPut(key, "changed val"));
-
-        Collection<Event> evts = g2.events().localQuery(F.<Event>alwaysTrue(), EVT_CACHE_REBALANCE_OBJECT_LOADED);
-
-        checkPreloadEvents(evts, g2, g2Keys);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 267de66..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,35 +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.distributed.near;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-
-/**
- *
- */
-public class GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest extends
-    GridCacheAtomicMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setLateAffinityAssignment(false);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 025b68b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,34 +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.distributed.near;
-
-import org.apache.ignite.configuration.IgniteConfiguration;
-
-/**
- *
- */
-public class GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest extends GridCachePartitionedMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setLateAffinityAssignment(false);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
index b512a64..c8ec304 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
@@ -76,7 +76,6 @@ public class IgnitePdsWholeClusterRestartTest extends GridCommonAbstractTest {
         ccfg1.setAffinity(new RendezvousAffinityFunction(false, 32));
         ccfg1.setBackups(2);
 
-        cfg.setLateAffinityAssignment(false);
         cfg.setActiveOnStart(false);
 
         // To avoid hostname lookup on start.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
index c74aada..303725f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
@@ -28,7 +28,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNea
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedCopyOnReadDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
@@ -45,7 +44,6 @@ public class GridActivationPartitionedCacheSuit extends GridActivationCacheAbstr
         addTest(GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.class);
         addTest(GridCachePartitionedCopyOnReadDisabledMultiNodeFullApiSelfTest.class);
         addTest(GridCachePartitionedFullApiSelfTest.class);
-        addTest(GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.class);
         addTest(GridCachePartitionedMultiNodeFullApiSelfTest.class);
         addTest(GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest.class);
         addTest(GridCachePartitionedNearDisabledFullApiSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 937a059..43069cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -134,8 +134,6 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
index d77296a..e811604 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
@@ -116,7 +116,6 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(false);
         cfg.setCacheConfiguration();
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 11a4a10..164ff6a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -36,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.CachePartiti
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicCopyOnReadDisabledMultiNodeFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeWithGroupFullApiSelfTest;
@@ -57,7 +56,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFilteredPutSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeCounterSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
@@ -155,10 +153,6 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicOnheapMultiNodeFullApiSelfTest.class);
 
-        // Old affinity assignment mode.
-        suite.addTestSuite(GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.class);
-
         // Multithreaded.
         suite.addTestSuite(GridCacheLocalFullApiMultithreadedSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 4c9accf..9ed7ee3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -78,7 +78,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtP
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtPreloadStartStopSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtPreloadUnloadSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledLockSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedPreloadEventsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedTopologyChangeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedUnloadEventsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheClearDuringRebalanceTest;
@@ -233,7 +232,6 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheAtomicNearEvictionEventSelfTest.class));
 
         suite.addTest(new TestSuite(GridCachePartitionedTopologyChangeSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedPreloadEventsSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedUnloadEventsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheColocatedOptimisticTransactionSelfTest.class));
         suite.addTestSuite(GridCacheAtomicMessageCountSelfTest.class);


[14/41] ignite git commit: IGNITE-5204: Fixed NPE on certain data with index inlining. This closes #2115.

Posted by sb...@apache.org.
IGNITE-5204: Fixed NPE on certain data with index inlining. This closes #2115.


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

Branch: refs/heads/ignite-5578-1
Commit: 0d6fb1ad73e8ed448dabe7c0cc631222835b52c4
Parents: ea4420e
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Fri Jul 7 15:54:52 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 7 15:54:52 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/database/InlineIndexHelper.java | 2 +-
 .../processors/query/h2/database/InlineIndexHelperTest.java      | 4 ++++
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0d6fb1ad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
index 1fd45f3..19cf857 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelper.java
@@ -441,7 +441,7 @@ public class InlineIndexHelper {
                     size = (short)s.length;
                 else {
                     s = trimUTF8(s, maxSize - 3);
-                    size = (short)(s.length | 0x8000);
+                    size = (short)(s == null ? 0 : s.length | 0x8000);
                 }
 
                 if (s == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0d6fb1ad/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
index 2579023..a2a3a72 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -167,6 +167,10 @@ public class InlineIndexHelperTest extends TestCase {
             assertTrue(ih.isValueFull(pageAddr, off));
 
             assertEquals("aaa", ih.get(pageAddr, off, 3 + 5).getString());
+
+            ih.put(pageAddr, off, ValueString.get("\u20acaaa"), 3 + 2);
+
+            assertNull(ih.get(pageAddr, off, 3 + 2));
         }
         finally {
             if (page != 0L)


[25/41] ignite git commit: .NET: Remove unused import and redundant cast

Posted by sb...@apache.org.
.NET: Remove unused import and redundant cast


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

Branch: refs/heads/ignite-5578-1
Commit: cb5ae9617af900e953e0f6f065adcd707d2b3830
Parents: 82e5f8a
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jul 7 20:15:47 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jul 7 20:15:47 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs  | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb5ae961/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index f5a5179..d08a191 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -27,7 +27,6 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Linq;
-    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cache.Affinity.Rendezvous;
@@ -242,7 +241,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
         private void Read(BinaryReader reader)
         {
             // Make sure system marshaller is used.
-            Debug.Assert(((BinaryReader) reader).Marshaller == BinaryUtils.Marshaller);
+            Debug.Assert(reader.Marshaller == BinaryUtils.Marshaller);
 
             AtomicityMode = (CacheAtomicityMode) reader.ReadInt();
             Backups = reader.ReadInt();


[35/41] ignite git commit: GG-12418 - WAL hangs on any error during segment rollover

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index 06bcf08..ca54e6f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -39,11 +39,12 @@ import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.events.WalSegmentArchivedEvent;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
-import org.apache.ignite.events.WalSegmentArchivedEvent;
 import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -177,7 +178,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
         final String consistentId = "127_0_0_1_47500";
-        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, WAL_SEGMENTS);
+
+        final FileIOFactory fileIOFactory = getConfiguration("").getPersistentStoreConfiguration().getFileIOFactory();
+        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, fileIOFactory, PAGE_SIZE, consistentId, WAL_SEGMENTS);
         final WALIterator it = mockItFactory.iterator(wal, walArchive);
         final int cntUsingMockIter = iterateAndCount(it);
 
@@ -188,7 +191,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final File walArchiveDirWithConsistentId = new File(walArchive, consistentId);
         final File walWorkDirWithConsistentId = new File(wal, consistentId);
 
-        final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, PAGE_SIZE);
+        final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, fileIOFactory, PAGE_SIZE);
         final int cntArchiveDir = iterateAndCount(factory.iteratorArchiveDirectory(walArchiveDirWithConsistentId));
 
         log.info("Total records loaded using directory : " + cntArchiveDir);

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
index 95079a0..ef162d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.jetbrains.annotations.Nullable;
 import org.mockito.Mockito;
@@ -42,6 +43,9 @@ public class MockWalIteratorFactory {
     /** Logger. */
     private final IgniteLogger log;
 
+    /** */
+    private final FileIOFactory ioFactory;
+
     /** Page size. */
     private final int pageSize;
 
@@ -58,8 +62,9 @@ public class MockWalIteratorFactory {
      * @param consistentId Consistent id.
      * @param segments Segments.
      */
-    public MockWalIteratorFactory(@Nullable IgniteLogger log, int pageSize, String consistentId, int segments) {
+    public MockWalIteratorFactory(@Nullable IgniteLogger log, FileIOFactory ioFactory, int pageSize, String consistentId, int segments) {
         this.log = log == null ? Mockito.mock(IgniteLogger.class) : log;
+        this.ioFactory = ioFactory;
         this.pageSize = pageSize;
         this.consistentId = consistentId;
         this.segments = segments;
@@ -80,6 +85,7 @@ public class MockWalIteratorFactory {
         when(persistentCfg1.getWalSegments()).thenReturn(segments);
         when(persistentCfg1.getTlbSize()).thenReturn(PersistentStoreConfiguration.DFLT_TLB_SIZE);
         when(persistentCfg1.getWalRecordIteratorBufferSize()).thenReturn(PersistentStoreConfiguration.DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE);
+        when(persistentCfg1.getFileIOFactory()).thenReturn(ioFactory);
 
         final IgniteConfiguration cfg = Mockito.mock(IgniteConfiguration.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index 8018705..4f4dedf 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageE
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFailoverTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.crc.IgniteDataIntegrityTests;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.reader.IgniteWalReaderTest;
@@ -65,6 +66,9 @@ public class IgnitePdsTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgnitePersistentStoreDataStructuresTest.class);
 
+        // Failover test
+        suite.addTestSuite(IgniteWalFlushFailoverTest.class);
+
         suite.addTestSuite(IgniteWalReaderTest.class);
         return suite;
     }


[26/41] ignite git commit: .NET: Fix IgniteConfigurationTest

Posted by sb...@apache.org.
.NET: Fix IgniteConfigurationTest


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

Branch: refs/heads/ignite-5578-1
Commit: 17904cb174a340ec928f4a7a4b12d294c2240d0f
Parents: cb5ae96
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jul 7 20:18:35 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jul 7 20:18:35 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/17904cb1/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 578d923..5e5cb1c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -683,7 +683,7 @@ namespace Apache.Ignite.Core.Tests
                         new MemoryPolicyConfiguration
                         {
                             Name = "myDefaultPlc",
-                            PageEvictionMode = DataPageEvictionMode.Random2Lru,
+                            PageEvictionMode = DataPageEvictionMode.Disabled,
                             InitialSize = 340 * 1024 * 1024,
                             MaxSize = 345 * 1024 * 1024,
                             EvictionThreshold = 0.88,
@@ -695,7 +695,7 @@ namespace Apache.Ignite.Core.Tests
                         new MemoryPolicyConfiguration
                         {
                             Name = "customPlc",
-                            PageEvictionMode = DataPageEvictionMode.RandomLru,
+                            PageEvictionMode = DataPageEvictionMode.Disabled,
                             MaxSize = 456 * 1024 * 1024,
                             EvictionThreshold = 0.77,
                             EmptyPagesPoolSize = 66,


[41/41] ignite git commit: ignite-5578 exchane future cleanup

Posted by sb...@apache.org.
ignite-5578 exchane future cleanup


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

Branch: refs/heads/ignite-5578-1
Commit: 5a38c46deb5b61bc3a32d7cc1b2b2e2190035a9f
Parents: f2568b7
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 15:00:00 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 15:00:00 2017 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      | 140 +++++++++++--------
 .../processors/cache/GridCachePreloader.java    |   8 +-
 .../cache/GridCachePreloaderAdapter.java        |   4 +-
 .../preloader/ForceRebalanceExchangeTask.java   |  58 ++++++++
 .../dht/preloader/GridDhtPartitionDemander.java |  22 +--
 .../preloader/GridDhtPartitionExchangeId.java   |  61 +++++++-
 .../GridDhtPartitionsExchangeFuture.java        | 136 +-----------------
 .../dht/preloader/GridDhtPreloader.java         |  33 ++---
 .../preloader/GridDhtPreloaderAssignments.java  |  21 ++-
 .../RebalanceReassignExchangeTask.java          |  44 ++++++
 10 files changed, 291 insertions(+), 236 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 f1db79a..1021e03 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
@@ -65,6 +65,7 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCach
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.ForceRebalanceExchangeTask;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
@@ -77,6 +78,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionsToReloadMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.RebalanceReassignExchangeTask;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
@@ -377,7 +379,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 "Node joined with smaller-than-local " +
                     "order [newOrder=" + n.order() + ", locOrder=" + loc.order() + ']';
 
-            exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+            exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
 
             exchFut = exchangeFuture(exchId, evt, cache,null, null);
         }
@@ -390,7 +392,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 ExchangeActions exchActions = stateChangeMsg.exchangeActions();
 
                 if (exchActions != null) {
-                    exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+                    exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
 
                     exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
                 }
@@ -401,7 +403,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 ExchangeActions exchActions = batch.exchangeActions();
 
                 if (exchActions != null) {
-                    exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+                    exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
 
                     exchFut = exchangeFuture(exchId, evt, cache, exchActions, null);
                 }
@@ -411,7 +413,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                 if (msg.exchangeId() == null) {
                     if (msg.exchangeNeeded()) {
-                        exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+                        exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
 
                         exchFut = exchangeFuture(exchId, evt, cache, null, msg);
                     }
@@ -422,7 +424,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             }
             else if (customMsg instanceof SnapshotDiscoveryMessage
                 && ((SnapshotDiscoveryMessage) customMsg).needExchange()) {
-                exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
+                exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt);
 
                 exchFut = exchangeFuture(exchId, evt, null, null, null);
             }
@@ -486,7 +488,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         assert discoEvt.topologyVersion() == startTopVer.topologyVersion();
 
-        return exchangeId(cctx.localNode().id(), startTopVer, EVT_NODE_JOINED);
+        return exchangeId(cctx.localNode().id(), startTopVer, discoEvt);
     }
 
     /**
@@ -863,27 +865,18 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
-     * @param exchFut Exchange future.
-     * @param reassign Dummy reassign flag.
+     * @param exchId Exchange ID.
      */
-    public void forceDummyExchange(boolean reassign,
-        GridDhtPartitionsExchangeFuture exchFut) {
-        exchWorker.addExchangeFuture(
-            new GridDhtPartitionsExchangeFuture(cctx, reassign, exchFut.discoveryEvent(), exchFut.exchangeId()));
+    public void forceReassign(GridDhtPartitionExchangeId exchId) {
+        exchWorker.forceReassign(exchId);
     }
 
     /**
-     * Forces preload exchange.
-     *
-     * @param exchFut Exchange future.
+     * @param exchId Exchange ID.
+     * @return Rebalance future.
      */
-    public IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionsExchangeFuture exchFut) {
-        GridCompoundFuture<Boolean, Boolean> fut = new GridCompoundFuture<>(CU.boolReducer());
-
-        exchWorker.addExchangeFuture(
-            new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut));
-
-        return fut;
+    public IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionExchangeId exchId) {
+        return exchWorker.forceRebalance(exchId);
     }
 
     /**
@@ -1203,10 +1196,10 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param nodeId Cause node ID.
      * @param topVer Topology version.
-     * @param evt Event type.
-     * @return Activity future ID.
+     * @param evt Event.
+     * @return Exchange ID instance.
      */
-    private GridDhtPartitionExchangeId exchangeId(UUID nodeId, AffinityTopologyVersion topVer, int evt) {
+    private GridDhtPartitionExchangeId exchangeId(UUID nodeId, AffinityTopologyVersion topVer, DiscoveryEvent evt) {
         return new GridDhtPartitionExchangeId(nodeId, evt, topVer);
     }
 
@@ -1677,7 +1670,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @return {@code True} if this is exchange task.
      */
     private static boolean isExchangeTask(CachePartitionExchangeWorkerTask task) {
-        return task instanceof GridDhtPartitionsExchangeFuture;
+        return task instanceof GridDhtPartitionsExchangeFuture ||
+            task instanceof RebalanceReassignExchangeTask ||
+            task instanceof ForceRebalanceExchangeTask;
     }
 
     /**
@@ -1787,13 +1782,32 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         }
 
         /**
+         * @param exchId Exchange ID.
+         */
+        void forceReassign(GridDhtPartitionExchangeId exchId) {
+            if (!hasPendingExchange() && !busy)
+                futQ.add(new RebalanceReassignExchangeTask(exchId));
+        }
+
+        /**
+         * @param exchId Exchange ID.
+         * @return Rebalance future.
+         */
+        IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionExchangeId exchId) {
+            GridCompoundFuture<Boolean, Boolean> fut = new GridCompoundFuture<>(CU.boolReducer());
+
+            futQ.add(new ForceRebalanceExchangeTask(exchId, fut));
+
+            return fut;
+        }
+
+        /**
          * @param exchFut Exchange future.
          */
         void addExchangeFuture(GridDhtPartitionsExchangeFuture exchFut) {
             assert exchFut != null;
 
-            if (!exchFut.dummy() || (!hasPendingExchange() && !busy))
-                futQ.offer(exchFut);
+            futQ.offer(exchFut);
 
             if (log.isDebugEnabled())
                 log.debug("Added exchange future to exchange worker: " + exchFut);
@@ -1922,22 +1936,37 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         continue;
                     }
 
-                    assert task instanceof GridDhtPartitionsExchangeFuture;
-
-                    GridDhtPartitionsExchangeFuture exchFut = (GridDhtPartitionsExchangeFuture)task;
-
                     busy = true;
 
                     Map<Integer, GridDhtPreloaderAssignments> assignsMap = null;
 
-                    boolean dummyReassign = exchFut.dummyReassign();
-                    boolean forcePreload = exchFut.forcePreload();
+                    boolean forcePreload = false;
+
+                    GridDhtPartitionExchangeId exchId;
+
+                    GridDhtPartitionsExchangeFuture exchFut = null;
 
                     try {
                         if (isCancelled())
                             break;
 
-                        if (!exchFut.dummy() && !exchFut.forcePreload()) {
+                        if (task instanceof RebalanceReassignExchangeTask) {
+                            exchId = ((RebalanceReassignExchangeTask) task).exchangeId();
+                        }
+                        else if (task instanceof ForceRebalanceExchangeTask) {
+                            forcePreload = true;
+
+                            timeout = 0; // Force refresh.
+
+                            exchId = ((ForceRebalanceExchangeTask)task).exchangeId();
+                        }
+                        else {
+                            assert task instanceof GridDhtPartitionsExchangeFuture : task;
+
+                            exchFut = (GridDhtPartitionsExchangeFuture)task;
+
+                            exchId = exchFut.exchangeId();
+
                             lastInitializedFut = exchFut;
 
                             exchFut.init();
@@ -2003,18 +2032,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             if (!cctx.kernalContext().clientNode() && changed && !hasPendingExchange())
                                 refreshPartitions();
                         }
-                        else {
-                            if (log.isDebugEnabled())
-                                log.debug("Got dummy exchange (will reassign)");
 
-                            if (!dummyReassign) {
-                                timeout = 0; // Force refresh.
-
-                                continue;
-                            }
-                        }
-
-                        if (!exchFut.skipPreload() ) {
+                        if (!cctx.kernalContext().clientNode()) {
                             assignsMap = new HashMap<>();
 
                             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
@@ -2024,7 +2043,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                                 // Don't delay for dummy reassigns to avoid infinite recursion.
                                 if (delay == 0 || forcePreload)
-                                    assigns = grp.preloader().assign(exchFut);
+                                    assigns = grp.preloader().assign(exchId, exchFut);
 
                                 assignsMap.put(grp.groupId(), assigns);
                             }
@@ -2059,6 +2078,11 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                         boolean assignsCancelled = false;
 
+                        GridCompoundFuture<Boolean, Boolean> forcedRebFut = null;
+
+                        if (task instanceof ForceRebalanceExchangeTask)
+                            forcedRebFut = ((ForceRebalanceExchangeTask)task).forcedRebalanceFuture();
+
                         for (Integer order : orderMap.descendingKeySet()) {
                             for (Integer grpId : orderMap.get(order)) {
                                 CacheGroupContext grp = cctx.cache().cacheGroup(grpId);
@@ -2077,7 +2101,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                     forcePreload,
                                     cnt,
                                     r,
-                                    exchFut.forcedRebalanceFuture());
+                                    forcedRebFut);
 
                                 if (cur != null) {
                                     rebList.add(grp.cacheOrGroupName());
@@ -2087,13 +2111,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             }
                         }
 
-                        if (exchFut.forcedRebalanceFuture() != null)
-                            exchFut.forcedRebalanceFuture().markInitialized();
+                        if (forcedRebFut != null)
+                            forcedRebFut.markInitialized();
 
                         if (assignsCancelled) { // Pending exchange.
                             U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
-                                "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                                "[top=" + exchId.topologyVersion() + ", evt=" + exchId.discoveryEventName() +
+                                ", node=" + exchId.nodeId() + ']');
                         }
                         else if (r != null) {
                             Collections.reverse(rebList);
@@ -2102,20 +2126,20 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                             if (!hasPendingExchange()) {
                                 U.log(log, "Rebalancing started " +
-                                    "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                    ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                                    "[top=" + exchId.topologyVersion() + ", evt=" + exchId.discoveryEventName() +
+                                    ", node=" + exchId.nodeId() + ']');
 
                                 r.run(); // Starts rebalancing routine.
                             }
                             else
                                 U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
-                                    "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                    ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                                    "[top=" + exchId.topologyVersion() + ", evt=" + exchId.discoveryEventName() +
+                                    ", node=" + exchId.nodeId() + ']');
                         }
                         else
                             U.log(log, "Skipping rebalancing (nothing scheduled) " +
-                                "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
-                                ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                                "[top=" + exchId.topologyVersion() + ", evt=" + exchId.discoveryEventName() +
+                                ", node=" + exchId.nodeId() + ']');
                     }
                 }
                 catch (IgniteInterruptedCheckedException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 4e74532..dc1624a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
@@ -63,10 +64,12 @@ public interface GridCachePreloader {
     public void onInitialExchangeComplete(@Nullable Throwable err);
 
     /**
-     * @param exchFut Exchange future to assign.
+     * @param exchId Exchange ID.
+     * @param exchFut Exchange future.
      * @return Assignments or {@code null} if detected that there are pending exchanges.
      */
-    @Nullable public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut);
+    @Nullable public GridDhtPreloaderAssignments assign(GridDhtPartitionExchangeId exchId,
+        @Nullable GridDhtPartitionsExchangeFuture exchFut);
 
     /**
      * Adds assignments to preloader.
@@ -75,6 +78,7 @@ public interface GridCachePreloader {
      * @param forcePreload Force preload flag.
      * @param cnt Counter.
      * @param next Runnable responsible for cache rebalancing start.
+     * @param forcedRebFut Rebalance future.
      * @return Rebalancing runnable.
      */
     public Runnable addAssignments(GridDhtPreloaderAssignments assignments,

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git 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
index d2ca229..c0accf6 100644
--- 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
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
@@ -152,7 +153,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
+    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionExchangeId exchId,
+        GridDhtPartitionsExchangeFuture exchFut) {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/ForceRebalanceExchangeTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/ForceRebalanceExchangeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/ForceRebalanceExchangeTask.java
new file mode 100644
index 0000000..c820175
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/ForceRebalanceExchangeTask.java
@@ -0,0 +1,58 @@
+/*
+ * 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.distributed.dht.preloader;
+
+import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+
+/**
+ *
+ */
+public class ForceRebalanceExchangeTask implements CachePartitionExchangeWorkerTask {
+    /** */
+    private final GridDhtPartitionExchangeId exchId;
+
+    /** */
+    private final GridCompoundFuture<Boolean, Boolean> forcedRebFut;
+
+    /**
+     * @param exchId Exchange ID.
+     * @param forcedRebFut Rebalance future.
+     */
+    public ForceRebalanceExchangeTask(GridDhtPartitionExchangeId exchId, GridCompoundFuture<Boolean, Boolean> forcedRebFut) {
+        assert exchId != null;
+        assert forcedRebFut != null;
+
+        this.exchId = exchId;
+        this.forcedRebFut = forcedRebFut;
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public GridDhtPartitionExchangeId exchangeId() {
+        return exchId;
+    }
+
+    /**
+     * @return Rebalance future.
+     */
+    public GridCompoundFuture<Boolean, Boolean> forcedRebalanceFuture() {
+        return forcedRebFut;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 4f34aba..248b739 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -189,7 +189,7 @@ public class GridDhtPartitionDemander {
     }
 
     /**
-     * Force Rebalance.
+     * @return Rebalance future.
      */
     IgniteInternalFuture<Boolean> forceRebalance() {
         GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
@@ -207,7 +207,7 @@ public class GridDhtPartitionDemander {
 
             exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                 @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                    IgniteInternalFuture<Boolean> fut0 = ctx.exchange().forceRebalance(exchFut);
+                    IgniteInternalFuture<Boolean> fut0 = ctx.exchange().forceRebalance(exchFut.exchangeId());
 
                     fut0.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
                         @Override public void apply(IgniteInternalFuture<Boolean> future) {
@@ -363,7 +363,7 @@ public class GridDhtPartitionDemander {
                 @Override public void onTimeout() {
                     exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
-                            ctx.exchange().forceRebalance(exchFut);
+                            ctx.exchange().forceRebalance(exchFut.exchangeId());
                         }
                     });
                 }
@@ -861,9 +861,9 @@ public class GridDhtPartitionDemander {
         /** Missed. */
         private final Map<UUID, Collection<Integer>> missed = new HashMap<>();
 
-        /** Exchange future. */
+        /** Exchange ID. */
         @GridToStringExclude
-        private final GridDhtPartitionsExchangeFuture exchFut;
+        private final GridDhtPartitionExchangeId exchId;
 
         /** Topology version. */
         private final AffinityTopologyVersion topVer;
@@ -884,7 +884,7 @@ public class GridDhtPartitionDemander {
             long updateSeq) {
             assert assigns != null;
 
-            exchFut = assigns.exchangeFuture();
+            exchId = assigns.exchangeId();
             topVer = assigns.topologyVersion();
 
             this.grp = grp;
@@ -898,7 +898,7 @@ public class GridDhtPartitionDemander {
          * Dummy future. Will be done by real one.
          */
         RebalanceFuture() {
-            this.exchFut = null;
+            this.exchId = null;
             this.topVer = null;
             this.ctx = null;
             this.grp = null;
@@ -1032,7 +1032,7 @@ public class GridDhtPartitionDemander {
                     return;
 
                 if (grp.eventRecordable(EVT_CACHE_REBALANCE_PART_LOADED))
-                    rebalanceEvent(p, EVT_CACHE_REBALANCE_PART_LOADED, exchFut.discoveryEvent());
+                    rebalanceEvent(p, EVT_CACHE_REBALANCE_PART_LOADED, exchId.discoveryEvent());
 
                 T2<Long, Collection<Integer>> t = remaining.get(nodeId);
 
@@ -1108,7 +1108,7 @@ public class GridDhtPartitionDemander {
 
                     onDone(false); //Finished but has missed partitions, will force dummy exchange
 
-                    ctx.exchange().forceDummyExchange(true, exchFut);
+                    ctx.exchange().forceReassign(exchId);
 
                     return;
                 }
@@ -1125,7 +1125,7 @@ public class GridDhtPartitionDemander {
          */
         private void sendRebalanceStartedEvent() {
             if (grp.eventRecordable(EVT_CACHE_REBALANCE_STARTED))
-                rebalanceEvent(EVT_CACHE_REBALANCE_STARTED, exchFut.discoveryEvent());
+                rebalanceEvent(EVT_CACHE_REBALANCE_STARTED, exchId.discoveryEvent());
         }
 
         /**
@@ -1133,7 +1133,7 @@ public class GridDhtPartitionDemander {
          */
         private void sendRebalanceFinishedEvent() {
             if (grp.eventRecordable(EVT_CACHE_REBALANCE_STOPPED))
-                rebalanceEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
+                rebalanceEvent(EVT_CACHE_REBALANCE_STOPPED, exchId.discoveryEvent());
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
index 012634e..0a49415 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java
@@ -23,6 +23,9 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import java.util.UUID;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -55,20 +58,27 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
     /** Topology version. */
     private AffinityTopologyVersion topVer;
 
+    /** */
+    @GridDirectTransient
+    private DiscoveryEvent discoEvt;
+
     /**
      * @param nodeId Node ID.
-     * @param evt Event.
+     * @param discoEvt Event.
      * @param topVer Topology version.
      */
-    public GridDhtPartitionExchangeId(UUID nodeId, int evt, @NotNull AffinityTopologyVersion topVer) {
+    public GridDhtPartitionExchangeId(UUID nodeId, DiscoveryEvent discoEvt, AffinityTopologyVersion topVer) {
         assert nodeId != null;
-        assert evt == EVT_NODE_LEFT || evt == EVT_NODE_FAILED || evt == EVT_NODE_JOINED ||
-            evt == EVT_DISCOVERY_CUSTOM_EVT;
-        assert topVer.topologyVersion() > 0;
+        assert topVer != null && topVer.topologyVersion() > 0 : topVer;
+        assert discoEvt != null;
 
         this.nodeId = nodeId;
-        this.evt = evt;
+        this.evt = discoEvt.type();
         this.topVer = topVer;
+        this.discoEvt = discoEvt;
+
+        assert evt == EVT_NODE_LEFT || evt == EVT_NODE_FAILED || evt == EVT_NODE_JOINED ||
+            evt == EVT_DISCOVERY_CUSTOM_EVT;
     }
 
     /**
@@ -93,6 +103,45 @@ public class GridDhtPartitionExchangeId implements Message, Comparable<GridDhtPa
     }
 
     /**
+     * @return Discovery event timestamp.
+     */
+    long eventTimestamp() {
+        assert discoEvt != null;
+
+        return discoEvt.timestamp();
+    }
+
+    /**
+     * @param discoEvt Discovery event.
+     */
+    void discoveryEvent(DiscoveryEvent discoEvt) {
+        this.discoEvt = discoEvt;
+    }
+
+    /**
+     * @return Discovery event.
+     */
+    DiscoveryEvent discoveryEvent() {
+        assert discoEvt != null;
+
+        return discoEvt;
+    }
+
+    /**
+     * @return Discovery event node.
+     */
+    public ClusterNode eventNode() {
+        return discoEvt.eventNode();
+    }
+
+    /**
+     * @return Discovery event name.
+     */
+    public String discoveryEventName() {
+        return U.gridEventName(evt);
+    }
+
+    /**
      * @return Order.
      */
     public AffinityTopologyVersion topologyVersion() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 21a3a13..6d98601 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
@@ -75,7 +75,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
 import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
-import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -110,14 +109,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     implements Comparable<GridDhtPartitionsExchangeFuture>, CachePartitionExchangeWorkerTask, IgniteDiagnosticAware {
     /** */
     public static final String EXCHANGE_LOG = "org.apache.ignite.internal.exchange.time";
-    /** Dummy flag. */
-    private final boolean dummy;
-
-    /** Force preload flag. */
-    private final boolean forcePreload;
-
-    /** Dummy reassign flag. */
-    private final boolean reassign;
 
     /** */
     @GridToStringExclude
@@ -196,9 +187,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     /** */
     private CacheAffinityChangeMessage affChangeMsg;
 
-    /** Skip preload flag. */
-    private boolean skipPreload;
-
     /** */
     private boolean clientOnlyExchange;
 
@@ -221,9 +209,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     @GridToStringExclude
     private volatile IgniteDhtPartitionHistorySuppliersMap partHistSuppliers = new IgniteDhtPartitionHistorySuppliersMap();
 
-    /** Forced Rebalance future. */
-    private GridCompoundFuture<Boolean, Boolean> forcedRebFut;
-
     /** */
     private volatile Map<Integer, Map<Integer, Long>> partHistReserved;
 
@@ -235,62 +220,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     private final AtomicBoolean done = new AtomicBoolean();
 
     /**
-     * Dummy future created to trigger reassignments if partition
-     * topology changed while preloading.
-     *
-     * @param cctx Cache context.
-     * @param reassign Dummy reassign flag.
-     * @param discoEvt Discovery event.
-     * @param exchId Exchange id.
-     */
-    public GridDhtPartitionsExchangeFuture(
-        GridCacheSharedContext cctx,
-        boolean reassign,
-        DiscoveryEvent discoEvt,
-        GridDhtPartitionExchangeId exchId
-    ) {
-        dummy = true;
-        forcePreload = false;
-
-        this.exchId = exchId;
-        this.reassign = reassign;
-        this.discoEvt = discoEvt;
-        this.cctx = cctx;
-
-        log = cctx.logger(getClass());
-        exchLog = cctx.logger(EXCHANGE_LOG);
-
-        onDone(exchId.topologyVersion());
-    }
-
-    /**
-     * Force preload future created to trigger reassignments if partition
-     * topology changed while preloading.
-     *
-     * @param cctx Cache context.
-     * @param discoEvt Discovery event.
-     * @param exchId Exchange id.
-     * @param forcedRebFut Forced Rebalance future.
-     */
-    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt,
-        GridDhtPartitionExchangeId exchId, GridCompoundFuture<Boolean, Boolean> forcedRebFut) {
-        dummy = false;
-        forcePreload = true;
-
-        this.exchId = exchId;
-        this.discoEvt = discoEvt;
-        this.cctx = cctx;
-        this.forcedRebFut = forcedRebFut;
-
-        log = cctx.logger(getClass());
-        exchLog = cctx.logger(EXCHANGE_LOG);
-
-        reassign = true;
-
-        onDone(exchId.topologyVersion());
-    }
-
-    /**
      * @param cctx Cache context.
      * @param busyLock Busy lock.
      * @param exchId Exchange ID.
@@ -309,10 +238,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         assert exchId.topologyVersion() != null;
         assert exchActions == null || !exchActions.empty();
 
-        dummy = false;
-        forcePreload = false;
-        reassign = false;
-
         this.cctx = cctx;
         this.busyLock = busyLock;
         this.exchId = exchId;
@@ -351,41 +276,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     * @return Skip preload flag.
-     */
-    public boolean skipPreload() {
-        return skipPreload;
-    }
-
-    /**
-     * @return Dummy flag.
-     */
-    public boolean dummy() {
-        return dummy;
-    }
-
-    /**
-     * @return Force preload flag.
-     */
-    public boolean forcePreload() {
-        return forcePreload;
-    }
-
-    /**
-     * @return Dummy reassign flag.
-     */
-    public boolean reassign() {
-        return reassign;
-    }
-
-    /**
-     * @return {@code True} if dummy reassign.
-     */
-    public boolean dummyReassign() {
-        return (dummy() || forcePreload()) && reassign();
-    }
-
-    /**
      * @param grpId Cache group ID.
      * @param partId Partition ID.
      * @return ID of history supplier node or null if it doesn't exist.
@@ -453,6 +343,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     public void onEvent(GridDhtPartitionExchangeId exchId, DiscoveryEvent discoEvt, DiscoCache discoCache) {
         assert exchId.equals(this.exchId);
 
+        this.exchId.discoveryEvent(discoEvt);
         this.discoEvt = discoEvt;
         this.discoCache = discoCache;
 
@@ -476,7 +367,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     /**
      * @return {@code True} if deactivate cluster exchange.
      */
-    boolean deactivateCluster() {
+    private boolean deactivateCluster() {
         return exchActions != null && exchActions.deactivate();
     }
 
@@ -495,13 +386,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     * @return Forced Rebalance future.
-     */
-    @Nullable public GridCompoundFuture<Boolean, Boolean> forcedRebalanceFuture() {
-        return forcedRebFut;
-    }
-
-    /**
      * @return {@code true} if entered to busy state.
      */
     private boolean enterBusy() {
@@ -538,7 +422,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         assert discoEvt != null : this;
         assert exchId.nodeId().equals(discoEvt.eventNode().id()) : this;
-        assert !dummy && !forcePreload : this;
 
         try {
             discoCache.updateAlives(cctx.discovery());
@@ -553,8 +436,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             boolean crdNode = crd != null && crd.isLocal();
 
-            skipPreload = cctx.kernalContext().clientNode();
-
             exchLog.info("Started exchange init [topVer=" + topVer +
                 ", crd=" + crdNode +
                 ", evt=" + discoEvt.type() +
@@ -1325,13 +1206,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
     /** {@inheritDoc} */
     @Override public boolean onDone(@Nullable AffinityTopologyVersion res, @Nullable Throwable err) {
-        boolean realExchange = !dummy && !forcePreload;
-
         if (!done.compareAndSet(false, true))
-            return dummy;
+            return false;
 
         if (err == null &&
-            realExchange &&
             !cctx.kernalContext().clientNode() &&
             (serverNodeDiscoveryEvent() || affChangeMsg != null)) {
             for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
@@ -1342,7 +1220,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             }
        }
 
-        if (err == null && realExchange) {
+        if (err == null) {
             if (centralizedAff) {
                 for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
                     if (grp.isLocal())
@@ -1412,14 +1290,14 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
         cctx.database().releaseHistoryForExchange();
 
-        if (err == null && realExchange) {
+        if (err == null) {
             for (CacheGroupContext grp : cctx.cache().cacheGroups()) {
                 if (!grp.isLocal())
                     grp.topology().onExchangeDone(grp.affinity().cachedAffinity(topologyVersion()));
             }
         }
 
-        if (super.onDone(res, err) && realExchange) {
+        if (super.onDone(res, err)) {
             if (log.isDebugEnabled())
                 log.debug("Completed partition exchange [localNode=" + cctx.localNodeId() + ", exchange= " + this +
                     ", durationFromInit=" + (U.currentTimeMillis() - initTs) + ']');
@@ -1441,7 +1319,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             return true;
         }
 
-        return dummy;
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 2b18c24..7efd4aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -179,22 +179,21 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
+    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionExchangeId exchId, GridDhtPartitionsExchangeFuture exchFut) {
         // No assignments for disabled preloader.
         GridDhtPartitionTopology top = grp.topology();
 
         if (!grp.rebalanceEnabled())
-            return new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
+            return new GridDhtPreloaderAssignments(exchId, top.topologyVersion());
 
         int partCnt = grp.affinity().partitions();
 
-        assert exchFut.forcePreload() || exchFut.dummyReassign() ||
-            exchFut.exchangeId().topologyVersion().equals(top.topologyVersion()) :
-            "Topology version mismatch [exchId=" + exchFut.exchangeId() +
+        assert exchFut == null || exchFut.topologyVersion().equals(top.topologyVersion()) :
+            "Topology version mismatch [exchId=" + exchId +
                 ", grp=" + grp.name() +
                 ", topVer=" + top.topologyVersion() + ']';
 
-        GridDhtPreloaderAssignments assigns = new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
+        GridDhtPreloaderAssignments assigns = new GridDhtPreloaderAssignments(exchId, top.topologyVersion());
 
         AffinityTopologyVersion topVer = assigns.topologyVersion();
 
@@ -204,7 +203,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
             if (ctx.exchange().hasPendingExchange()) {
                 if (log.isDebugEnabled())
                     log.debug("Skipping assignments creation, exchange worker has pending assignments: " +
-                        exchFut.exchangeId());
+                        exchId);
 
                 assigns.cancelled(true);
 
@@ -220,7 +219,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
                 ClusterNode histSupplier = null;
 
-                if (ctx.database().persistenceEnabled()) {
+                if (ctx.database().persistenceEnabled() && exchFut != null) {
                     UUID nodeId = exchFut.partitionHistorySupplier(grp.groupId(), p);
 
                     if (nodeId != null)
@@ -243,7 +242,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                     if (msg == null) {
                         assigns.put(histSupplier, msg = new GridDhtPartitionDemandMessage(
                             top.updateSequence(),
-                            exchFut.exchangeId().topologyVersion(),
+                            exchId.topologyVersion(),
                             grp.groupId()));
                     }
 
@@ -292,14 +291,12 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                         top.own(part);
 
                         if (grp.eventRecordable(EVT_CACHE_REBALANCE_PART_DATA_LOST)) {
-                            DiscoveryEvent discoEvt = exchFut.discoveryEvent();
-
-                        grp.addRebalanceEvent(p,
-                            EVT_CACHE_REBALANCE_PART_DATA_LOST,
-                            discoEvt.eventNode(),
-                            discoEvt.type(),
-                            discoEvt.timestamp());
-                    }
+                            grp.addRebalanceEvent(p,
+                                EVT_CACHE_REBALANCE_PART_DATA_LOST,
+                                exchId.eventNode(),
+                                exchId.event(),
+                                exchId.eventTimestamp());
+                        }
 
                         if (log.isDebugEnabled())
                             log.debug("Owning partition as there are no other owners: " + part);
@@ -312,7 +309,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                     if (msg == null) {
                         assigns.put(n, msg = new GridDhtPartitionDemandMessage(
                             top.updateSequence(),
-                            exchFut.exchangeId().topologyVersion(),
+                            exchId.topologyVersion(),
                             grp.groupId()));
                     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
index 3f82c9b..41dd076 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
@@ -30,25 +30,24 @@ public class GridDhtPreloaderAssignments extends ConcurrentHashMap<ClusterNode,
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Exchange future. */
-    @GridToStringExclude
-    private final GridDhtPartitionsExchangeFuture exchFut;
+    /** */
+    private final GridDhtPartitionExchangeId exchangeId;
 
-    /** Last join order. */
+    /** */
     private final AffinityTopologyVersion topVer;
 
     /** */
     private boolean cancelled;
 
     /**
-     * @param exchFut Exchange future.
+     * @param exchangeId Exchange ID.
      * @param topVer Last join order.
      */
-    public GridDhtPreloaderAssignments(GridDhtPartitionsExchangeFuture exchFut, AffinityTopologyVersion topVer) {
-        assert exchFut != null;
+    public GridDhtPreloaderAssignments(GridDhtPartitionExchangeId exchangeId, AffinityTopologyVersion topVer) {
+        assert exchangeId != null;
         assert topVer.topologyVersion() > 0 : topVer;
 
-        this.exchFut = exchFut;
+        this.exchangeId = exchangeId;
         this.topVer = topVer;
     }
 
@@ -69,8 +68,8 @@ public class GridDhtPreloaderAssignments extends ConcurrentHashMap<ClusterNode,
     /**
      * @return Exchange future.
      */
-    GridDhtPartitionsExchangeFuture exchangeFuture() {
-        return exchFut;
+    GridDhtPartitionExchangeId exchangeId() {
+        return exchangeId;
     }
 
     /**
@@ -82,7 +81,7 @@ public class GridDhtPreloaderAssignments extends ConcurrentHashMap<ClusterNode,
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(GridDhtPreloaderAssignments.class, this, "exchId", exchFut.exchangeId(),
+        return S.toString(GridDhtPreloaderAssignments.class, this, "exchId", exchangeId,
             "super", super.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a38c46d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/RebalanceReassignExchangeTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/RebalanceReassignExchangeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/RebalanceReassignExchangeTask.java
new file mode 100644
index 0000000..9a76a8e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/RebalanceReassignExchangeTask.java
@@ -0,0 +1,44 @@
+/*
+ * 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.distributed.dht.preloader;
+
+import org.apache.ignite.internal.processors.cache.CachePartitionExchangeWorkerTask;
+
+/**
+ *
+ */
+public class RebalanceReassignExchangeTask implements CachePartitionExchangeWorkerTask {
+    /** */
+    private final GridDhtPartitionExchangeId exchId;
+
+    /**
+     * @param exchId Exchange ID.
+     */
+    public RebalanceReassignExchangeTask(GridDhtPartitionExchangeId exchId) {
+        assert exchId != null;
+
+        this.exchId = exchId;
+    }
+
+    /**
+     * @return Exchange ID.
+     */
+    public GridDhtPartitionExchangeId exchangeId() {
+        return exchId;
+    }
+}


[33/41] ignite git commit: GridDhtPartitionsExchangeFuture minor: removed duplicated updateAlives call.

Posted by sb...@apache.org.
GridDhtPartitionsExchangeFuture minor: removed duplicated updateAlives call.


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

Branch: refs/heads/ignite-5578-1
Commit: 313f86e5536f82cf34a9a95ad16fa1afbe37a63e
Parents: b085fa0
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 13:01:02 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 13:01:02 2017 +0300

----------------------------------------------------------------------
 .../distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/313f86e5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 90c8aaf..21a3a13 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
@@ -545,8 +545,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
             AffinityTopologyVersion topVer = topologyVersion();
 
-            discoCache.updateAlives(cctx.discovery());
-
             srvNodes = new ArrayList<>(discoCache.serverNodes());
 
             remaining.addAll(F.nodeIds(F.view(srvNodes, F.remoteNodes(cctx.localNodeId()))));


[24/41] ignite git commit: IGNITE-5716 .NET: Fix 2-byte field offset handling

Posted by sb...@apache.org.
IGNITE-5716 .NET: Fix 2-byte field offset handling


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

Branch: refs/heads/ignite-5578-1
Commit: 82e5f8a6553323e793c01c54e24dda6d47188ce6
Parents: eb37f53
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jul 7 19:28:54 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jul 7 19:28:54 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Binary/BinaryFooterTest.cs                  | 146 +++++++++++++++++++
 .../Binary/BinarySelfTest.cs                    |  32 ----
 .../Impl/Binary/BinaryObject.cs                 |   2 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   2 +-
 .../Impl/Binary/BinaryObjectSchemaField.cs      |   3 +
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |  93 +++++++++---
 .../Impl/Binary/BinaryReader.cs                 |  49 +------
 8 files changed, 228 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 09eac70..d91f0e6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -72,6 +72,7 @@
     <Compile Include="Binary\BinaryDateTimeTest.cs" />
     <Compile Include="Binary\BinaryEqualityComparerTest.cs" />
     <Compile Include="Binary\BinaryBuilderSelfTestDynamicRegistration.cs" />
+    <Compile Include="Binary\BinaryFooterTest.cs" />
     <Compile Include="Binary\BinaryNameMapperTest.cs" />
     <Compile Include="Binary\BinaryReaderWriterTest.cs" />
     <Compile Include="Binary\BinarySelfTestSimpleName.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
new file mode 100644
index 0000000..36f2f65
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
@@ -0,0 +1,146 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using System;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests binary footer integrity (field offsets).
+    /// Writes objects of various sizes to test schema compaction 
+    /// (where field offsets can be stored as 1, 2 or 4 bytes).
+    /// </summary>
+    public class BinaryFooterTest
+    {
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests full footers in offline mode.
+        /// </summary>
+        [Test]
+        public void TestFullFooterOffline()
+        {
+            // Register type to avoid unregistered type name in binary object.
+            // Use new marshaller to read and write to avoid schema caching.
+
+            TestOffsets(() => new Marshaller(new BinaryConfiguration(typeof(OffsetTest))
+            {
+                // Compact footers do not work in offline mode.
+                CompactFooter = false
+            }));
+        }
+
+        /// <summary>
+        /// Tests the full footer online.
+        /// </summary>
+        [Test]
+        public void TestFullFooterOnline()
+        {
+            var ignite = Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                BinaryConfiguration = new BinaryConfiguration
+                {
+                    CompactFooter = false
+                }
+            });
+
+            TestOffsets(() => ((Ignite) ignite).Marshaller);
+        }
+
+        /// <summary>
+        /// Tests the full footer online.
+        /// </summary>
+        [Test]
+        public void TestCompactFooterOnline()
+        {
+            var ignite = Ignition.Start(TestUtils.GetTestConfiguration());
+
+            TestOffsets(() => ((Ignite) ignite).Marshaller);
+        }
+
+        /// <summary>
+        /// Tests the offsets.
+        /// </summary>
+        private static void TestOffsets(Func<Marshaller> getMarsh)
+        {
+            // Corner cases are byte/sbyte/short/ushort max values.
+            foreach (var i in new[] {1, sbyte.MaxValue, byte.MaxValue, short.MaxValue, ushort.MaxValue})
+            {
+                foreach (var j in new[] {-1, 0, 1})
+                {
+                    var arrSize = i + j;
+
+                    var dt = new OffsetTest
+                    {
+                        Arr = Enumerable.Range(0, arrSize).Select(x => (byte) x).ToArray(),
+                        Int = arrSize
+                    };
+
+                    var bytes = getMarsh().Marshal(dt);
+
+                    var res = getMarsh().Unmarshal<OffsetTest>(bytes);
+                    var binRes = getMarsh().Unmarshal<IBinaryObject>(bytes, BinaryMode.ForceBinary);
+                    var binFieldRes = new OffsetTest
+                    {
+                        Arr = binRes.GetField<byte[]>("arr"),
+                        Int = binRes.GetField<int>("int")
+                    };
+
+                    foreach (var r in new[] {res, binRes.Deserialize<OffsetTest>(), binFieldRes})
+                    {
+                        Assert.AreEqual(dt.Arr, r.Arr);
+                        Assert.AreEqual(dt.Int, r.Int);
+                    }
+                }
+            }
+        }
+
+        /// <summary>
+        /// Offset test.
+        /// </summary>
+        private class OffsetTest : IBinarizable
+        {
+            public byte[] Arr;  // Array to enforce field offset.
+            public int Int;     // Value at offset.
+
+            public void WriteBinary(IBinaryWriter writer)
+            {
+                writer.WriteByteArray("arr", Arr);
+                writer.WriteInt("int", Int);
+            }
+
+            public void ReadBinary(IBinaryReader reader)
+            {
+                // Read in different order to enforce full schema scan.
+                Int = reader.ReadInt("int");
+                Arr = reader.ReadByteArray("arr");
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
index 2b22d5a..e24dca0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinarySelfTest.cs
@@ -1534,38 +1534,6 @@ namespace Apache.Ignite.Core.Tests.Binary
             Assert.AreEqual(nDateArr, obj2.NDateArr);
         }
 
-        /// <summary>
-        /// Writes objects of various sizes to test schema compaction 
-        /// (where field offsets can be stored as 1, 2 or 4 bytes).
-        /// </summary>
-        [Test]
-        public void TestCompactSchema()
-        {
-            var marsh = new Marshaller(new BinaryConfiguration
-            {
-                TypeConfigurations = new List<BinaryTypeConfiguration>
-                {
-                    new BinaryTypeConfiguration(typeof (SpecialArray)),
-                    new BinaryTypeConfiguration(typeof (SpecialArrayMarshalAware))
-                }
-            });
-
-            var dt = new SpecialArrayMarshalAware();
-
-            foreach (var i in new[] {1, 5, 10, 13, 14, 15, 100, 200, 1000, 5000, 15000, 30000})
-            {
-                dt.NGuidArr = Enumerable.Range(1, i).Select(x => (Guid?) Guid.NewGuid()).ToArray();
-                dt.NDateArr = Enumerable.Range(1, i).Select(x => (DateTime?) DateTime.Now.AddDays(x)).ToArray();
-
-                var bytes = marsh.Marshal(dt);
-
-                var res = marsh.Unmarshal<SpecialArrayMarshalAware>(bytes);
-
-                CollectionAssert.AreEquivalent(dt.NGuidArr, res.NGuidArr);
-                CollectionAssert.AreEquivalent(dt.NDateArr, res.NDateArr);
-            }
-        }
-
         [Test]
         public void TestBinaryConfigurationValidation()
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
index 8c5cee6..370233f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
@@ -239,7 +239,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             {
                 var hdr = BinaryObjectHeader.Read(stream, _offset);
 
-                _fields = BinaryObjectSchemaSerializer.ReadSchema(stream, _offset, hdr, desc.Schema,_marsh)
+                _fields = BinaryObjectSchemaSerializer.ReadSchema(stream, _offset, hdr, desc.Schema, _marsh.Ignite)
                     .ToDictionary() ?? EmptyFields;
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 91fe12a..c310b3a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -617,7 +617,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     {
                         // New object, write in full form.
                         var inSchema = BinaryObjectSchemaSerializer.ReadSchema(inStream, inStartPos, inHeader, 
-                            _desc.Schema, _binary.Marshaller);
+                            _desc.Schema, _binary.Marshaller.Ignite);
 
                         var outSchema = BinaryObjectSchemaHolder.Current;
                         var schemaIdx = outSchema.PushSchema();

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaField.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaField.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaField.cs
index 3c5339a..be6278a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaField.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaField.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Impl.Binary
 {
+    using System.Diagnostics;
     using System.Runtime.InteropServices;
 
     /// <summary>
@@ -41,6 +42,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="offset">The offset.</param>
         public BinaryObjectSchemaField(int id, int offset)
         {
+            Debug.Assert(offset >= 0);
+
             Id = id;
             Offset = offset;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
index e2f9ea7..1d699c2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
@@ -54,18 +54,17 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="position">The position.</param>
         /// <param name="hdr">The header.</param>
         /// <param name="schema">The schema.</param>
-        /// <param name="marsh">The marshaller.</param>
+        /// <param name="ignite">The ignite.</param>
         /// <returns>
         /// Schema.
         /// </returns>
         public static BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position, BinaryObjectHeader hdr, 
-            BinaryObjectSchema schema, Marshaller marsh)
+            BinaryObjectSchema schema, Ignite ignite)
         {
             Debug.Assert(stream != null);
             Debug.Assert(schema != null);
-            Debug.Assert(marsh != null);
 
-            return ReadSchema(stream, position, hdr, () => GetFieldIds(hdr, schema, marsh));
+            return ReadSchema(stream, position, hdr, () => GetFieldIds(hdr, schema, ignite));
         }
 
         /// <summary>
@@ -78,8 +77,8 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <returns>
         /// Schema.
         /// </returns>
-        public static BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position, BinaryObjectHeader hdr, 
-            Func<int[]> fieldIdsFunc)
+        public static unsafe BinaryObjectSchemaField[] ReadSchema(IBinaryStream stream, int position, 
+            BinaryObjectHeader hdr, Func<int[]> fieldIdsFunc)
         {
             Debug.Assert(stream != null);
             Debug.Assert(fieldIdsFunc != null);
@@ -110,7 +109,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 else if (offsetSize == 2)
                 {
                     for (var i = 0; i < schemaSize; i++)
-                        res[i] = new BinaryObjectSchemaField(fieldIds[i], stream.ReadShort());
+                        res[i] = new BinaryObjectSchemaField(fieldIds[i], (ushort) stream.ReadShort());
                 }
                 else
                 {
@@ -128,12 +127,22 @@ namespace Apache.Ignite.Core.Impl.Binary
                 else if (offsetSize == 2)
                 {
                     for (var i = 0; i < schemaSize; i++)
-                        res[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadShort());
+                        res[i] = new BinaryObjectSchemaField(stream.ReadInt(), (ushort) stream.ReadShort());
                 }
                 else
                 {
-                    for (var i = 0; i < schemaSize; i++)
-                        res[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadInt());
+                    if (BitConverter.IsLittleEndian)
+                    {
+                        fixed (BinaryObjectSchemaField* ptr = &res[0])
+                        {
+                            stream.Read((byte*) ptr, schemaSize * BinaryObjectSchemaField.Size);
+                        }
+                    }
+                    else
+                    {
+                        for (var i = 0; i < schemaSize; i++)
+                            res[i] = new BinaryObjectSchemaField(stream.ReadInt(), stream.ReadInt());
+                    }
                 }
             }
 
@@ -220,7 +229,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     {
                         fixed (BinaryObjectSchemaField* ptr = &fields[offset])
                         {
-                            stream.Write((byte*)ptr, count / BinaryObjectSchemaField.Size);
+                            stream.Write((byte*)ptr, count * BinaryObjectSchemaField.Size);
                         }
                     }
                     else
@@ -243,22 +252,66 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <summary>
         /// Gets the field ids.
         /// </summary>
-        private static int[] GetFieldIds(BinaryObjectHeader hdr, BinaryObjectSchema schema, Marshaller marsh)
+        private static int[] GetFieldIds(BinaryObjectHeader hdr, Ignite ignite)
         {
-            var fieldIds = schema.Get(hdr.SchemaId);
+            Debug.Assert(hdr.TypeId != BinaryUtils.TypeUnregistered);
+
+            int[] fieldIds = null;
+
+            if (ignite != null)
+            {
+                fieldIds = ignite.BinaryProcessor.GetSchema(hdr.TypeId, hdr.SchemaId);
+            }
 
             if (fieldIds == null)
             {
-                Debug.Assert(hdr.TypeId != BinaryUtils.TypeUnregistered);
+                throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
+                                                "typeId=" + hdr.TypeId + ", schemaId=" + hdr.SchemaId + ']');
+            }
+            return fieldIds;
+        }
 
-                if (marsh.Ignite != null)
-                    fieldIds = marsh.Ignite.BinaryProcessor.GetSchema(hdr.TypeId, hdr.SchemaId);
+        /// <summary>
+        /// Reads the schema, maintains stream position.
+        /// </summary>
+        public static int[] GetFieldIds(BinaryObjectHeader hdr, Ignite ignite, IBinaryStream stream, int objectPos)
+        {
+            Debug.Assert(stream != null);
 
-                if (fieldIds == null)
-                    throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
-                                                    "typeId=" + hdr.TypeId + ", schemaId=" + hdr.SchemaId + ']');
+            if (hdr.IsCompactFooter)
+            {
+                // Get schema from Java
+                return GetFieldIds(hdr, ignite);
             }
-            return fieldIds;
+
+            var pos = stream.Position;
+
+            stream.Seek(objectPos + hdr.SchemaOffset, SeekOrigin.Begin);
+
+            var count = hdr.SchemaFieldCount;
+
+            var offsetSize = hdr.SchemaFieldOffsetSize;
+
+            var res = new int[count];
+
+            for (var i = 0; i < count; i++)
+            {
+                res[i] = stream.ReadInt();
+                stream.Seek(offsetSize, SeekOrigin.Current);  // Skip offsets.
+            }
+
+            stream.Seek(pos, SeekOrigin.Begin);
+
+            return res;
+        }
+
+
+        /// <summary>
+        /// Gets the field ids.
+        /// </summary>
+        private static int[] GetFieldIds(BinaryObjectHeader hdr, BinaryObjectSchema schema, Ignite ignite)
+        {
+            return schema.Get(hdr.SchemaId) ?? GetFieldIds(hdr, ignite);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/82e5f8a6/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index e792dce..76237c4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -20,7 +20,6 @@ namespace Apache.Ignite.Core.Impl.Binary
     using System;
     using System.Collections;
     using System.Collections.Generic;
-    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using Apache.Ignite.Core.Binary;
@@ -783,7 +782,8 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 if (_frame.Schema == null)
                 {
-                    _frame.Schema = ReadSchema(desc.TypeId);
+                    _frame.Schema = 
+                        BinaryObjectSchemaSerializer.GetFieldIds(_frame.Hdr, Marshaller.Ignite, Stream, _frame.Pos);
 
                     desc.Schema.Add(_frame.Hdr.SchemaId, _frame.Schema);
                 }
@@ -795,49 +795,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Reads the schema.
-        /// </summary>
-        private int[] ReadSchema(int typeId)
-        {
-            if (_frame.Hdr.IsCompactFooter)
-            {
-                // Get schema from Java
-                var ignite = Marshaller.Ignite;
-
-                Debug.Assert(typeId != BinaryUtils.TypeUnregistered);
-
-                var schema = ignite == null
-                    ? null
-                    : ignite.BinaryProcessor.GetSchema(_frame.Hdr.TypeId, _frame.Hdr.SchemaId);
-
-                if (schema == null)
-                    throw new BinaryObjectException("Cannot find schema for object with compact footer [" +
-                        "typeId=" + typeId + ", schemaId=" + _frame.Hdr.SchemaId + ']');
-
-                return schema;
-            }
-
-            var pos = Stream.Position;
-
-            Stream.Seek(_frame.Pos + _frame.Hdr.SchemaOffset, SeekOrigin.Begin);
-
-            var count = _frame.Hdr.SchemaFieldCount;
-
-            var offsetSize = _frame.Hdr.SchemaFieldOffsetSize;
-
-            var res = new int[count];
-
-            for (int i = 0; i < count; i++)
-            {
-                res[i] = Stream.ReadInt();
-                Stream.Seek(offsetSize, SeekOrigin.Current);
-            }
-
-            Stream.Seek(pos, SeekOrigin.Begin);
-
-            return res;
-        }
-        /// <summary>
         /// Reads the handle object.
         /// </summary>
         private T ReadHandleObject<T>(int pos, Type typeOverride)
@@ -942,7 +899,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 int pos;
 
-                if (!_frame.SchemaMap.TryGetValue(fieldId, out pos))
+                if (_frame.SchemaMap == null || !_frame.SchemaMap.TryGetValue(fieldId, out pos))
                     return false;
 
                 Stream.Seek(pos + _frame.Pos, SeekOrigin.Begin);


[39/41] ignite git commit: ignite-5075 Rename cacheId -> groupId

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index 4a56ec7..af20136 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -269,8 +269,8 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void read(int cacheId, long pageId, ByteBuffer pageBuf) throws IgniteCheckedException {
-        read(cacheId, pageId, pageBuf, false);
+    @Override public void read(int grpId, long pageId, ByteBuffer pageBuf) throws IgniteCheckedException {
+        read(grpId, pageId, pageBuf, false);
     }
 
     /**
@@ -296,20 +296,20 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void readHeader(int cacheId, int partId, ByteBuffer buf) throws IgniteCheckedException {
-        PageStore store = getStore(cacheId, partId);
+    @Override public void readHeader(int grpId, int partId, ByteBuffer buf) throws IgniteCheckedException {
+        PageStore store = getStore(grpId, partId);
 
         store.readHeader(buf);
     }
 
     /** {@inheritDoc} */
-    @Override public void write(int cacheId, long pageId, ByteBuffer pageBuf,int tag) throws IgniteCheckedException {
-        writeInternal(cacheId, pageId, pageBuf, tag);
+    @Override public void write(int grpId, long pageId, ByteBuffer pageBuf, int tag) throws IgniteCheckedException {
+        writeInternal(grpId, pageId, pageBuf, tag);
     }
 
     /** {@inheritDoc} */
-    @Override public long pageOffset(int cacheId, long pageId) throws IgniteCheckedException {
-        PageStore store = getStore(cacheId, PageIdUtils.partId(pageId));
+    @Override public long pageOffset(int grpId, long pageId) throws IgniteCheckedException {
+        PageStore store = getStore(grpId, PageIdUtils.partId(pageId));
 
         return store.pageOffset(pageId);
     }
@@ -447,20 +447,20 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public void sync(int cacheId, int partId) throws IgniteCheckedException {
-        getStore(cacheId, partId).sync();
+    @Override public void sync(int grpId, int partId) throws IgniteCheckedException {
+        getStore(grpId, partId).sync();
     }
 
     /** {@inheritDoc} */
-    @Override public void ensure(int cacheId, int partId) throws IgniteCheckedException {
-        getStore(cacheId, partId).ensure();
+    @Override public void ensure(int grpId, int partId) throws IgniteCheckedException {
+        getStore(grpId, partId).ensure();
     }
 
     /** {@inheritDoc} */
-    @Override public long allocatePage(int cacheId, int partId, byte flags) throws IgniteCheckedException {
+    @Override public long allocatePage(int grpId, int partId, byte flags) throws IgniteCheckedException {
         assert partId <= PageIdAllocator.MAX_PARTITION_ID || partId == PageIdAllocator.INDEX_PARTITION;
 
-        PageStore store = getStore(cacheId, partId);
+        PageStore store = getStore(grpId, partId);
 
         long pageIdx = store.allocatePage();
 
@@ -468,13 +468,13 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
-    @Override public long metaPageId(final int cacheId) {
+    @Override public long metaPageId(final int grpId) {
         return metaPageId;
     }
 
     /** {@inheritDoc} */
-    @Override public int pages(int cacheId, int partId) throws IgniteCheckedException {
-        PageStore store = getStore(cacheId, partId);
+    @Override public int pages(int grpId, int partId) throws IgniteCheckedException {
+        PageStore store = getStore(grpId, partId);
 
         return store.pages();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
index 844bc02..e99a5ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
@@ -203,7 +203,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 PageUtils.getBytes(pageAddr, data.offset(), payload, 0, rowSize);
 
                 wal.log(new DataPageInsertRecord(
-                    cacheId,
+                    grpId,
                     pageId,
                     payload));
             }
@@ -246,7 +246,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                 PageUtils.getBytes(pageAddr, data.offset(), payload, 0, payloadSize);
 
-                wal.log(new DataPageInsertFragmentRecord(cacheId, pageId, payload, lastLink));
+                wal.log(new DataPageInsertFragmentRecord(grpId, pageId, payload, lastLink));
             }
 
             return written + payloadSize;
@@ -448,7 +448,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         assert part <= PageIdAllocator.MAX_PARTITION_ID;
         assert part != PageIdAllocator.INDEX_PARTITION;
 
-        return pageMem.allocatePage(cacheId, part, PageIdAllocator.FLAG_DATA);
+        return pageMem.allocatePage(grpId, part, PageIdAllocator.FLAG_DATA);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
index eb5df6b..39a6865 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
@@ -333,7 +333,7 @@ public abstract class PagesList extends DataStructure {
                     io.resetCount(pageAddr);
 
                     if (needWalDeltaRecord(pageId, page, null))
-                        wal.log(new PageListMetaResetCountRecord(cacheId, pageId));
+                        wal.log(new PageListMetaResetCountRecord(grpId, pageId));
 
                     nextPageId = io.getNextMetaPageId(pageAddr);
                 }
@@ -682,13 +682,13 @@ public abstract class PagesList extends DataStructure {
             incrementBucketSize(bucket);
 
             if (needWalDeltaRecord(pageId, page, null))
-                wal.log(new PagesListAddPageRecord(cacheId, pageId, dataId));
+                wal.log(new PagesListAddPageRecord(grpId, pageId, dataId));
 
             DataPageIO dataIO = DataPageIO.VERSIONS.forPage(dataAddr);
             dataIO.setFreeListPageId(dataAddr, pageId);
 
             if (needWalDeltaRecord(dataId, dataPage, null))
-                wal.log(new DataPageSetFreeListPageRecord(cacheId, dataId, pageId));
+                wal.log(new DataPageSetFreeListPageRecord(grpId, dataId, pageId));
         }
 
         return true;
@@ -728,11 +728,11 @@ public abstract class PagesList extends DataStructure {
             setupNextPage(io, pageId, pageAddr, newDataId, dataAddr);
 
             if (needWalDeltaRecord(pageId, page, null))
-                wal.log(new PagesListSetNextRecord(cacheId, pageId, newDataId));
+                wal.log(new PagesListSetNextRecord(grpId, pageId, newDataId));
 
             if (needWalDeltaRecord(dataId, data, null))
                 wal.log(new PagesListInitNewPageRecord(
-                    cacheId,
+                    grpId,
                     dataId,
                     io.getType(),
                     io.getVersion(),
@@ -761,13 +761,13 @@ public abstract class PagesList extends DataStructure {
                     setupNextPage(io, pageId, pageAddr, nextId, nextPageAddr);
 
                     if (needWalDeltaRecord(pageId, page, null))
-                        wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
+                        wal.log(new PagesListSetNextRecord(grpId, pageId, nextId));
 
                     int idx = io.addPage(nextPageAddr, dataId, pageSize());
 
                     if (needWalDeltaRecord(nextId, nextPage, nextWalPlc))
                         wal.log(new PagesListInitNewPageRecord(
-                            cacheId,
+                            grpId,
                             nextId,
                             io.getType(),
                             io.getVersion(),
@@ -781,7 +781,7 @@ public abstract class PagesList extends DataStructure {
                     dataIO.setFreeListPageId(dataAddr, nextId);
 
                     if (needWalDeltaRecord(dataId, data, null))
-                        wal.log(new DataPageSetFreeListPageRecord(cacheId, dataId, nextId));
+                        wal.log(new DataPageSetFreeListPageRecord(grpId, dataId, nextId));
 
                     incrementBucketSize(bucket);
 
@@ -853,12 +853,12 @@ public abstract class PagesList extends DataStructure {
                         setupNextPage(io, prevId, prevAddr, nextId, nextPageAddr);
 
                         if (needWalDeltaRecord(prevId, prevPage, walPlc))
-                            wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
+                            wal.log(new PagesListSetNextRecord(grpId, prevId, nextId));
 
                         // Here we should never write full page, because it is known to be new.
                         if (needWalDeltaRecord(nextId, nextPage, FALSE))
                             wal.log(new PagesListInitNewPageRecord(
-                                cacheId,
+                                grpId,
                                 nextId,
                                 io.getType(),
                                 io.getVersion(),
@@ -887,7 +887,7 @@ public abstract class PagesList extends DataStructure {
                 else {
                     // TODO: use single WAL record for bag?
                     if (needWalDeltaRecord(prevId, prevPage, walPlc))
-                        wal.log(new PagesListAddPageRecord(cacheId, prevId, nextId));
+                        wal.log(new PagesListAddPageRecord(grpId, prevId, nextId));
 
                     incrementBucketSize(bucket);
                 }
@@ -1035,7 +1035,7 @@ public abstract class PagesList extends DataStructure {
                         decrementBucketSize(bucket);
 
                         if (needWalDeltaRecord(tailId, tailPage, null))
-                            wal.log(new PagesListRemovePageRecord(cacheId, tailId, pageId));
+                            wal.log(new PagesListRemovePageRecord(grpId, tailId, pageId));
 
                         dirty = true;
 
@@ -1085,7 +1085,7 @@ public abstract class PagesList extends DataStructure {
                             initIo.initNewPage(tailAddr, dataPageId, pageSize());
 
                             if (needWalDeltaRecord(tailId, tailPage, null)) {
-                                wal.log(new InitNewPageRecord(cacheId, tailId, initIo.getType(),
+                                wal.log(new InitNewPageRecord(grpId, tailId, initIo.getType(),
                                     initIo.getVersion(), dataPageId));
                             }
                         }
@@ -1163,13 +1163,13 @@ public abstract class PagesList extends DataStructure {
                 decrementBucketSize(bucket);
 
                 if (needWalDeltaRecord(pageId, page, null))
-                    wal.log(new PagesListRemovePageRecord(cacheId, pageId, dataId));
+                    wal.log(new PagesListRemovePageRecord(grpId, pageId, dataId));
 
                 // Reset free list page ID.
                 dataIO.setFreeListPageId(dataAddr, 0L);
 
                 if (needWalDeltaRecord(dataId, dataPage, null))
-                    wal.log(new DataPageSetFreeListPageRecord(cacheId, dataId, 0L));
+                    wal.log(new DataPageSetFreeListPageRecord(grpId, dataId, 0L));
 
                 if (!io.isEmpty(pageAddr))
                     return true; // In optimistic case we still have something in the page and can leave it as is.
@@ -1343,7 +1343,7 @@ public abstract class PagesList extends DataStructure {
                 nextIO.setPreviousId(nextAddr, 0);
 
                 if (needWalDeltaRecord(nextId, nextPage, null))
-                    wal.log(new PagesListSetPreviousRecord(cacheId, nextId, 0L));
+                    wal.log(new PagesListSetPreviousRecord(grpId, nextId, 0L));
             }
             else // Do a fair merge: link previous and next to each other.
                 fairMerge(prevId, pageId, nextId, nextPage, nextAddr);
@@ -1384,12 +1384,12 @@ public abstract class PagesList extends DataStructure {
                 prevIO.setNextId(prevAddr, nextId);
 
                 if (needWalDeltaRecord(prevId, prevPage, null))
-                    wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
+                    wal.log(new PagesListSetNextRecord(grpId, prevId, nextId));
 
                 nextIO.setPreviousId(nextAddr, prevId);
 
                 if (needWalDeltaRecord(nextId, nextPage, null))
-                    wal.log(new PagesListSetPreviousRecord(cacheId, nextId, prevId));
+                    wal.log(new PagesListSetPreviousRecord(grpId, nextId, prevId));
             }
             finally {
                 writeUnlock(prevId, prevPage, prevAddr, true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTable.java
index 78d83b3..19d26ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/FullPageIdTable.java
@@ -45,13 +45,13 @@ public class FullPageIdTable {
     private static final long EMPTY_PAGE_ID = EMPTY_FULL_PAGE_ID.pageId();
 
     /** */
-    private static final int EMPTY_CACHE_ID = EMPTY_FULL_PAGE_ID.cacheId();
+    private static final int EMPTY_CACHE_GRP_ID = EMPTY_FULL_PAGE_ID.groupId();
 
     /** */
     private static final long REMOVED_PAGE_ID = 0x8000000000000000L;
 
     /** */
-    private static final int REMOVED_CACHE_ID = 0;
+    private static final int REMOVED_CACHE_GRP_ID = 0;
 
     /** */
     private static final int EQUAL = 0;
@@ -178,13 +178,13 @@ public class FullPageIdTable {
     /**
      * Removes key-value association for the given key.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      */
-    public void remove(int cacheId, long pageId, int tag) {
-        assert assertKey(cacheId, pageId);
+    public void remove(int grpId, long pageId, int tag) {
+        assert assertKey(grpId, pageId);
 
-        int index = removeKey(cacheId, pageId, tag);
+        int index = removeKey(grpId, pageId, tag);
 
         if (index >= 0)
             setValueAt(index, 0);
@@ -225,18 +225,18 @@ public class FullPageIdTable {
     public long clearAt(int idx, GridPredicate3<Integer, Long, Integer> pred, long absent) {
         long base = entryBase(idx);
 
-        int cacheId = GridUnsafe.getInt(base);
+        int grpId = GridUnsafe.getInt(base);
         int tag = GridUnsafe.getInt(base + 4);
         long pageId = GridUnsafe.getLong(base + 8);
 
-        if ((pageId == REMOVED_PAGE_ID && cacheId == REMOVED_CACHE_ID)
-            || (pageId == EMPTY_PAGE_ID && cacheId == EMPTY_CACHE_ID))
+        if ((pageId == REMOVED_PAGE_ID && grpId == REMOVED_CACHE_GRP_ID)
+            || (pageId == EMPTY_PAGE_ID && grpId == EMPTY_CACHE_GRP_ID))
             return absent;
 
-        if (pred.apply(cacheId, pageId, tag)) {
+        if (pred.apply(grpId, pageId, tag)) {
             long res = valueAt(idx);
 
-            setKeyAt(idx, REMOVED_CACHE_ID, REMOVED_PAGE_ID);
+            setKeyAt(idx, REMOVED_CACHE_GRP_ID, REMOVED_PAGE_ID);
             setValueAt(idx, 0);
 
             return res;
@@ -367,7 +367,7 @@ public class FullPageIdTable {
         while (++step <= maxSteps);
 
         if (foundIndex != -1) {
-            setKeyAt(foundIndex, REMOVED_CACHE_ID, REMOVED_PAGE_ID);
+            setKeyAt(foundIndex, REMOVED_CACHE_GRP_ID, REMOVED_PAGE_ID);
 
             decrementSize();
         }
@@ -382,17 +382,17 @@ public class FullPageIdTable {
     private int testKeyAt(int index, int testCacheId, long testPageId, int testTag) {
         long base = entryBase(index);
 
-        int cacheId = GridUnsafe.getInt(base);
+        int grpId = GridUnsafe.getInt(base);
         int tag = GridUnsafe.getInt(base + 4);
         long pageId = GridUnsafe.getLong(base + 8);
 
-        if (pageId == REMOVED_PAGE_ID && cacheId == REMOVED_CACHE_ID)
+        if (pageId == REMOVED_PAGE_ID && grpId == REMOVED_CACHE_GRP_ID)
             return REMOVED;
-        else if (pageId == testPageId && cacheId == testCacheId && tag >= testTag)
+        else if (pageId == testPageId && grpId == testCacheId && tag >= testTag)
             return EQUAL;
-        else if (pageId == testPageId && cacheId == testCacheId && tag < testTag)
+        else if (pageId == testPageId && grpId == testCacheId && tag < testTag)
             return OUTDATED;
-        else if (pageId == EMPTY_PAGE_ID && cacheId == EMPTY_CACHE_ID)
+        else if (pageId == EMPTY_PAGE_ID && grpId == EMPTY_CACHE_GRP_ID)
             return EMPTY;
         else
             return NOT_EQUAL;
@@ -405,34 +405,34 @@ public class FullPageIdTable {
     private boolean isValuePresentAt(final int idx) {
         long base = entryBase(idx);
 
-        int cacheId = GridUnsafe.getInt(base);
+        int grpId = GridUnsafe.getInt(base);
         long pageId = GridUnsafe.getLong(base + 8);
 
-        return !((pageId == REMOVED_PAGE_ID && cacheId == REMOVED_CACHE_ID)
-            || (pageId == EMPTY_PAGE_ID && cacheId == EMPTY_CACHE_ID));
+        return !((pageId == REMOVED_PAGE_ID && grpId == REMOVED_CACHE_GRP_ID)
+            || (pageId == EMPTY_PAGE_ID && grpId == EMPTY_CACHE_GRP_ID));
     }
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @return {@code True} if checks succeeded.
      */
-    private boolean assertKey(int cacheId, long pageId) {
-        assert cacheId != EMPTY_CACHE_ID && PageIdUtils.isEffectivePageId(pageId):
-            "cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId);
+    private boolean assertKey(int grpId, long pageId) {
+        assert grpId != EMPTY_CACHE_GRP_ID && PageIdUtils.isEffectivePageId(pageId):
+            "grpId=" + grpId + ", pageId=" + U.hexLong(pageId);
 
         return true;
     }
 
     /**
      * @param index Entry index.
-     * @param cacheId Cache ID to write.
+     * @param grpId Cache group ID to write.
      * @param pageId Page ID to write.
      */
-    private void setKeyAt(int index, int cacheId, long pageId) {
+    private void setKeyAt(int index, int grpId, long pageId) {
         long base = entryBase(index);
 
-        GridUnsafe.putInt(base, cacheId);
+        GridUnsafe.putInt(base, grpId);
         GridUnsafe.putLong(base + 8, pageId);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
index 3246f21..7c63d41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
@@ -77,13 +77,13 @@ public interface PageMemoryEx extends PageMemory {
      * @see #acquirePage(int, long)
      * Will not read page from file if it is not present in memory
      *
-     * @param cacheId Cache id.
+     * @param grpId Cache group ID.
      * @param pageId Page id.
      * @param restore Get page for restore
      * @throws IgniteCheckedException If failed.
      * @return Page.
      */
-    public long acquirePage(int cacheId, long pageId, boolean restore) throws IgniteCheckedException;
+    public long acquirePage(int grpId, long pageId, boolean restore) throws IgniteCheckedException;
 
     /**
      * Heuristic method which allows a thread to check if it safe to start memory struture modifications

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 6bb5c33..ab6619d 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -714,7 +714,7 @@ public class PageMemoryImpl implements PageMemoryEx {
 
                             if (curPage != null
                                 && deltaRecord.pageId() == fullId.pageId()
-                                && deltaRecord.cacheId() == fullId.cacheId()) {
+                                && deltaRecord.groupId() == fullId.groupId()) {
                                 assert tmpAddr != null;
 
                                 deltaRecord.applyDelta(this, tmpAddr);
@@ -727,8 +727,8 @@ public class PageMemoryImpl implements PageMemoryEx {
 
             if (restored == null)
                 throw new AssertionError(String.format(
-                    "Page is broken. Can't restore it from WAL. (cacheId = %d, pageId = %X).",
-                    fullId.cacheId(), fullId.pageId()
+                    "Page is broken. Can't restore it from WAL. (grpId = %d, pageId = %X).",
+                    fullId.groupId(), fullId.pageId()
                 ));
 
             long pageAddr = writeLockPage(absPtr, fullId, false);
@@ -796,7 +796,7 @@ public class PageMemoryImpl implements PageMemoryEx {
     @Override public Integer getForCheckpoint(FullPageId fullId, ByteBuffer tmpBuf, CheckpointMetricsTracker tracker) {
         assert tmpBuf.remaining() == pageSize();
 
-        Segment seg = segment(fullId.cacheId(), fullId.pageId());
+        Segment seg = segment(fullId.groupId(), fullId.pageId());
 
         long absPtr = 0;
 
@@ -807,10 +807,10 @@ public class PageMemoryImpl implements PageMemoryEx {
         seg.readLock().lock();
 
         try {
-            tag = seg.partTag(fullId.cacheId(), PageIdUtils.partId(fullId.pageId()));
+            tag = seg.partTag(fullId.groupId(), PageIdUtils.partId(fullId.pageId()));
 
             relPtr = seg.loadedPages.get(
-                fullId.cacheId(),
+                fullId.groupId(),
                 PageIdUtils.effectivePageId(fullId.pageId()),
                 tag,
                 INVALID_REL_PTR,
@@ -839,10 +839,10 @@ public class PageMemoryImpl implements PageMemoryEx {
             try {
                 // Double-check.
                 relPtr = seg.loadedPages.get(
-                    fullId.cacheId(),
+                    fullId.groupId(),
                     PageIdUtils.effectivePageId(fullId.pageId()),
                     seg.partTag(
-                        fullId.cacheId(),
+                        fullId.groupId(),
                         PageIdUtils.partId(fullId.pageId())
                     ),
                     INVALID_REL_PTR,
@@ -855,7 +855,7 @@ public class PageMemoryImpl implements PageMemoryEx {
                 if (relPtr == OUTDATED_REL_PTR) {
                     relPtr = refreshOutdatedPage(
                         seg,
-                        fullId.cacheId(),
+                        fullId.groupId(),
                         PageIdUtils.effectivePageId(fullId.pageId()),
                         true
                     );
@@ -1245,7 +1245,7 @@ public class PageMemoryImpl implements PageMemoryEx {
      * @return {@code True} if it was added to the checkpoint list.
      */
     boolean isInCheckpoint(FullPageId pageId) {
-        Segment seg = segment(pageId.cacheId(), pageId.pageId());
+        Segment seg = segment(pageId.groupId(), pageId.pageId());
 
         Collection<FullPageId> pages0 = seg.segCheckpointPages;
 
@@ -1256,7 +1256,7 @@ public class PageMemoryImpl implements PageMemoryEx {
      * @param fullPageId Page ID to clear.
      */
     void clearCheckpoint(FullPageId fullPageId) {
-        Segment seg = segment(fullPageId.cacheId(), fullPageId.pageId());
+        Segment seg = segment(fullPageId.groupId(), fullPageId.pageId());
 
         Collection<FullPageId> pages0 = seg.segCheckpointPages;
 
@@ -1300,14 +1300,14 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         if (dirty) {
             if (!wasDirty || forceAdd) {
-                boolean added = segment(pageId.cacheId(), pageId.pageId()).dirtyPages.add(pageId);
+                boolean added = segment(pageId.groupId(), pageId.pageId()).dirtyPages.add(pageId);
 
                 if (added)
                     memMetrics.incrementDirtyPages();
             }
         }
         else {
-            boolean rmv = segment(pageId.cacheId(), pageId.pageId()).dirtyPages.remove(pageId);
+            boolean rmv = segment(pageId.groupId(), pageId.pageId()).dirtyPages.remove(pageId);
 
             if (rmv)
                 memMetrics.decrementDirtyPages();
@@ -1330,12 +1330,12 @@ public class PageMemoryImpl implements PageMemoryEx {
     }
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @return Segment.
      */
-    private Segment segment(int cacheId, long pageId) {
-        int idx = segmentIndex(cacheId, pageId, segments.length);
+    private Segment segment(int grpId, long pageId) {
+        int idx = segmentIndex(grpId, pageId, segments.length);
 
         return segments[idx];
     }
@@ -1344,11 +1344,11 @@ public class PageMemoryImpl implements PageMemoryEx {
      * @param pageId Page ID.
      * @return Segment index.
      */
-    public static int segmentIndex(int cacheId, long pageId, int segments) {
+    public static int segmentIndex(int grpId, long pageId, int segments) {
         pageId = PageIdUtils.effectivePageId(pageId);
 
         // Take a prime number larger than total number of partitions.
-        int hash = U.hash(pageId * 65537 + cacheId);
+        int hash = U.hash(pageId * 65537 + grpId);
 
         return U.safeAbs(hash) % segments;
     }
@@ -1645,7 +1645,7 @@ public class PageMemoryImpl implements PageMemoryEx {
             assert writeLock().isHeldByCurrentThread();
 
             // Do not evict cache meta pages.
-            if (fullPageId.pageId() == storeMgr.metaPageId(fullPageId.cacheId()))
+            if (fullPageId.pageId() == storeMgr.metaPageId(fullPageId.groupId()))
                 return false;
 
             if (PageHeader.isAcquired(absPtr))
@@ -1662,7 +1662,7 @@ public class PageMemoryImpl implements PageMemoryEx {
                         fullPageId,
                         wrapPointer(absPtr + PAGE_OVERHEAD, pageSize()),
                         partTag(
-                            fullPageId.cacheId(),
+                            fullPageId.groupId(),
                             PageIdUtils.partId(fullPageId.pageId())
                         )
                     );
@@ -1743,10 +1743,10 @@ public class PageMemoryImpl implements PageMemoryEx {
                     assert fullId.equals(nearest.fullId()) : "Invalid page mapping [tableId=" + nearest.fullId() +
                         ", actual=" + fullId + ", nearest=" + nearest;
 
-                    boolean outdated = partTag < partTag(fullId.cacheId(), PageIdUtils.partId(fullId.pageId()));
+                    boolean outdated = partTag < partTag(fullId.groupId(), PageIdUtils.partId(fullId.pageId()));
 
                     if (outdated)
-                        return refreshOutdatedPage(this, fullId.cacheId(), fullId.pageId(), true);
+                        return refreshOutdatedPage(this, fullId.groupId(), fullId.pageId(), true);
 
                     boolean pinned = PageHeader.isAcquired(absPageAddr);
 
@@ -1797,10 +1797,10 @@ public class PageMemoryImpl implements PageMemoryEx {
                 }
 
                 loadedPages.remove(
-                    fullPageId.cacheId(),
+                    fullPageId.groupId(),
                     PageIdUtils.effectivePageId(fullPageId.pageId()),
                     partTag(
-                        fullPageId.cacheId(),
+                        fullPageId.groupId(),
                         PageIdUtils.partId(fullPageId.pageId())
                     )
                 );
@@ -1834,8 +1834,8 @@ public class PageMemoryImpl implements PageMemoryEx {
 
                 FullPageId fullId = PageHeader.fullPageId(absPageAddr);
 
-                if (partTag < partTag(fullId.cacheId(), PageIdUtils.partId(fullId.pageId())))
-                    return refreshOutdatedPage(this, fullId.cacheId(), fullId.pageId(), true);
+                if (partTag < partTag(fullId.groupId(), PageIdUtils.partId(fullId.pageId())))
+                    return refreshOutdatedPage(this, fullId.groupId(), fullId.pageId(), true);
 
                 boolean pinned = PageHeader.isAcquired(absPageAddr);
 
@@ -1851,10 +1851,10 @@ public class PageMemoryImpl implements PageMemoryEx {
 
                 if (prepareEvict(fullPageId, absEvictAddr)) {
                     loadedPages.remove(
-                        fullPageId.cacheId(),
+                        fullPageId.groupId(),
                         PageIdUtils.effectivePageId(fullPageId.pageId()),
                         partTag(
-                            fullPageId.cacheId(),
+                            fullPageId.groupId(),
                             PageIdUtils.partId(fullPageId.pageId())
                         )
                     );
@@ -1888,14 +1888,14 @@ public class PageMemoryImpl implements PageMemoryEx {
         }
 
         /**
-         * @param cacheId Cache ID.
+         * @param grpId Cache group ID.
          * @param partId Partition ID.
          * @return Partition tag.
          */
-        private int partTag(int cacheId, int partId) {
+        private int partTag(int grpId, int partId) {
             assert getReadHoldCount() > 0 || getWriteHoldCount() > 0;
 
-            Integer tag = partTagMap.get(new T2<>(cacheId, partId));
+            Integer tag = partTagMap.get(new T2<>(grpId, partId));
 
             if (tag == null)
                 return 1;
@@ -1904,14 +1904,14 @@ public class PageMemoryImpl implements PageMemoryEx {
         }
 
         /**
-         * @param cacheId Cache ID.
+         * @param grpId Cache group ID.
          * @param partId Partition ID.
          * @return New partition tag.
          */
-        private int incrementPartTag(int cacheId, int partId) {
+        private int incrementPartTag(int grpId, int partId) {
             assert getWriteHoldCount() > 0;
 
-            T2<Integer, Integer> t = new T2<>(cacheId, partId);
+            T2<Integer, Integer> t = new T2<>(grpId, partId);
 
             Integer tag = partTagMap.get(t);
 
@@ -1921,7 +1921,7 @@ public class PageMemoryImpl implements PageMemoryEx {
                 return 2;
             }
             else if (tag == Integer.MAX_VALUE) {
-                U.warn(log, "Partition tag overflow [cacheId=" + cacheId + ", partId=" + partId + "]");
+                U.warn(log, "Partition tag overflow [grpId=" + grpId + ", partId=" + partId + "]");
 
                 partTagMap.put(t, 0);
 
@@ -2224,7 +2224,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         private static void fullPageId(final long absPtr, final FullPageId fullPageId) {
             pageId(absPtr, fullPageId.pageId());
 
-            pageCacheId(absPtr, fullPageId.cacheId());
+            pageCacheId(absPtr, fullPageId.groupId());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 74d251a..c73b4c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -377,7 +377,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             byte[] newRowBytes = io.store(pageAddr, idx, newRow, null, needWal);
 
             if (needWal)
-                wal.log(new ReplaceRecord<>(cacheId, pageId, io, newRowBytes, idx));
+                wal.log(new ReplaceRecord<>(grpId, pageId, io, newRowBytes, idx));
 
             return FOUND;
         }
@@ -2200,7 +2200,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         io.splitExistingPage(pageAddr, mid, fwdId);
 
         if (needWalDeltaRecord(pageId, page, null))
-            wal.log(new SplitExistingPageRecord(cacheId, pageId, mid, fwdId));
+            wal.log(new SplitExistingPageRecord(grpId, pageId, mid, fwdId));
 
         return res;
     }
@@ -2685,7 +2685,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             byte[] rowBytes = io.insert(pageAddr, idx, row, null, rightId, needWal);
 
             if (needWal)
-                wal.log(new InsertRecord<>(cacheId, pageId, io, idx, rowBytes, rightId));
+                wal.log(new InsertRecord<>(grpId, pageId, io, idx, rowBytes, rightId));
         }
 
         /**
@@ -2728,7 +2728,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             inner(io).setLeft(fwdPageAddr, 0, rightId);
 
                             if (needWalDeltaRecord(fwdId, fwdPage, fwdPageWalPlc)) // Rare case, we can afford separate WAL record to avoid complexity.
-                                wal.log(new FixLeftmostChildRecord(cacheId, fwdId, rightId));
+                                wal.log(new FixLeftmostChildRecord(grpId, fwdId, rightId));
                         }
                     }
                     else // Insert into newly allocated forward page.
@@ -2744,7 +2744,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         io.setCount(pageAddr, cnt - 1);
 
                         if (needWalDeltaRecord(pageId, page, null)) // Rare case, we can afford separate WAL record to avoid complexity.
-                            wal.log(new FixCountRecord(cacheId, pageId, cnt - 1));
+                            wal.log(new FixCountRecord(grpId, pageId, cnt - 1));
                     }
 
                     if (!hadFwd && lvl == getRootLevel()) { // We are splitting root.
@@ -2775,7 +2775,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                                     needWal);
 
                                 if (needWal)
-                                    wal.log(new NewRootInitRecord<>(cacheId, newRootId, newRootId,
+                                    wal.log(new NewRootInitRecord<>(grpId, newRootId, newRootId,
                                         inner(io), pageId, moveUpRowBytes, fwdId));
                             }
                             finally {
@@ -3656,7 +3656,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             io.remove(pageAddr, idx, cnt);
 
             if (needWalDeltaRecord(pageId, page, walPlc))
-                wal.log(new RemoveRecord(cacheId, pageId, idx, cnt));
+                wal.log(new RemoveRecord(grpId, pageId, idx, cnt));
         }
 
         /**
@@ -3909,7 +3909,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             leaf.io.setRemoveId(leaf.buf, rmvId);
 
             if (needWalDeltaRecord(leaf.pageId, leaf.page, leaf.walPlc))
-                wal.log(new FixRemoveId(cacheId, leaf.pageId, rmvId));
+                wal.log(new FixRemoveId(grpId, leaf.pageId, rmvId));
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
index 0a7b3dd..663fe0c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV1Serializer.java
@@ -166,7 +166,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGE_RECORD:
                 PageSnapshot snap = (PageSnapshot)record;
 
-                buf.putInt(snap.fullPageId().cacheId());
+                buf.putInt(snap.fullPageId().groupId());
                 buf.putLong(snap.fullPageId().pageId());
                 buf.put(snap.pageData());
 
@@ -182,7 +182,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PARTITION_DESTROY:
                 PartitionDestroyRecord partDestroy = (PartitionDestroyRecord)record;
 
-                buf.putInt(partDestroy.cacheId());
+                buf.putInt(partDestroy.groupId());
                 buf.putInt(partDestroy.partitionId());
 
                 break;
@@ -190,7 +190,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case META_PAGE_INIT:
                 MetaPageInitRecord updRootsRec = (MetaPageInitRecord)record;
 
-                buf.putInt(updRootsRec.cacheId());
+                buf.putInt(updRootsRec.groupId());
                 buf.putLong(updRootsRec.pageId());
 
                 buf.putShort((short)updRootsRec.ioType());
@@ -203,7 +203,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PARTITION_META_PAGE_UPDATE_COUNTERS:
                 MetaPageUpdatePartitionDataRecord partDataRec = (MetaPageUpdatePartitionDataRecord)record;
 
-                buf.putInt(partDataRec.cacheId());
+                buf.putInt(partDataRec.groupId());
                 buf.putLong(partDataRec.pageId());
 
                 buf.putLong(partDataRec.updateCounter());
@@ -261,7 +261,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case DATA_PAGE_INSERT_RECORD:
                 DataPageInsertRecord diRec = (DataPageInsertRecord)record;
 
-                buf.putInt(diRec.cacheId());
+                buf.putInt(diRec.groupId());
                 buf.putLong(diRec.pageId());
 
                 buf.putShort((short)diRec.payload().length);
@@ -273,7 +273,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case DATA_PAGE_UPDATE_RECORD:
                 DataPageUpdateRecord uRec = (DataPageUpdateRecord)record;
 
-                buf.putInt(uRec.cacheId());
+                buf.putInt(uRec.groupId());
                 buf.putLong(uRec.pageId());
                 buf.putInt(uRec.itemId());
 
@@ -286,7 +286,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case DATA_PAGE_INSERT_FRAGMENT_RECORD:
                 final DataPageInsertFragmentRecord difRec = (DataPageInsertFragmentRecord)record;
 
-                buf.putInt(difRec.cacheId());
+                buf.putInt(difRec.groupId());
                 buf.putLong(difRec.pageId());
 
                 buf.putLong(difRec.lastLink());
@@ -298,7 +298,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case DATA_PAGE_REMOVE_RECORD:
                 DataPageRemoveRecord drRec = (DataPageRemoveRecord)record;
 
-                buf.putInt(drRec.cacheId());
+                buf.putInt(drRec.groupId());
                 buf.putLong(drRec.pageId());
 
                 buf.put((byte)drRec.itemId());
@@ -308,7 +308,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case DATA_PAGE_SET_FREE_LIST_PAGE:
                 DataPageSetFreeListPageRecord freeListRec = (DataPageSetFreeListPageRecord)record;
 
-                buf.putInt(freeListRec.cacheId());
+                buf.putInt(freeListRec.groupId());
                 buf.putLong(freeListRec.pageId());
 
                 buf.putLong(freeListRec.freeListPage());
@@ -318,7 +318,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case INIT_NEW_PAGE_RECORD:
                 InitNewPageRecord inpRec = (InitNewPageRecord)record;
 
-                buf.putInt(inpRec.cacheId());
+                buf.putInt(inpRec.groupId());
                 buf.putLong(inpRec.pageId());
 
                 buf.putShort((short)inpRec.ioType());
@@ -330,7 +330,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_META_PAGE_INIT_ROOT:
                 MetaPageInitRootRecord imRec = (MetaPageInitRootRecord)record;
 
-                buf.putInt(imRec.cacheId());
+                buf.putInt(imRec.groupId());
                 buf.putLong(imRec.pageId());
 
                 buf.putLong(imRec.rootId());
@@ -340,7 +340,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_META_PAGE_INIT_ROOT2:
                 MetaPageInitRootInlineRecord imRec2 = (MetaPageInitRootInlineRecord)record;
 
-                buf.putInt(imRec2.cacheId());
+                buf.putInt(imRec2.groupId());
                 buf.putLong(imRec2.pageId());
 
                 buf.putLong(imRec2.rootId());
@@ -351,7 +351,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_META_PAGE_ADD_ROOT:
                 MetaPageAddRootRecord arRec = (MetaPageAddRootRecord)record;
 
-                buf.putInt(arRec.cacheId());
+                buf.putInt(arRec.groupId());
                 buf.putLong(arRec.pageId());
 
                 buf.putLong(arRec.rootId());
@@ -361,7 +361,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_META_PAGE_CUT_ROOT:
                 MetaPageCutRootRecord crRec = (MetaPageCutRootRecord)record;
 
-                buf.putInt(crRec.cacheId());
+                buf.putInt(crRec.groupId());
                 buf.putLong(crRec.pageId());
 
                 break;
@@ -369,7 +369,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_INIT_NEW_ROOT:
                 NewRootInitRecord<?> riRec = (NewRootInitRecord<?>)record;
 
-                buf.putInt(riRec.cacheId());
+                buf.putInt(riRec.groupId());
                 buf.putLong(riRec.pageId());
 
                 buf.putLong(riRec.rootId());
@@ -385,7 +385,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_PAGE_RECYCLE:
                 RecycleRecord recRec = (RecycleRecord)record;
 
-                buf.putInt(recRec.cacheId());
+                buf.putInt(recRec.groupId());
                 buf.putLong(recRec.pageId());
 
                 buf.putLong(recRec.newPageId());
@@ -395,7 +395,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_PAGE_INSERT:
                 InsertRecord<?> inRec = (InsertRecord<?>)record;
 
-                buf.putInt(inRec.cacheId());
+                buf.putInt(inRec.groupId());
                 buf.putLong(inRec.pageId());
 
                 buf.putShort((short)inRec.io().getType());
@@ -410,7 +410,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_FIX_LEFTMOST_CHILD:
                 FixLeftmostChildRecord flRec = (FixLeftmostChildRecord)record;
 
-                buf.putInt(flRec.cacheId());
+                buf.putInt(flRec.groupId());
                 buf.putLong(flRec.pageId());
 
                 buf.putLong(flRec.rightId());
@@ -420,7 +420,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_FIX_COUNT:
                 FixCountRecord fcRec = (FixCountRecord)record;
 
-                buf.putInt(fcRec.cacheId());
+                buf.putInt(fcRec.groupId());
                 buf.putLong(fcRec.pageId());
 
                 buf.putShort((short)fcRec.count());
@@ -430,7 +430,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_PAGE_REPLACE:
                 ReplaceRecord<?> rRec = (ReplaceRecord<?>)record;
 
-                buf.putInt(rRec.cacheId());
+                buf.putInt(rRec.groupId());
                 buf.putLong(rRec.pageId());
 
                 buf.putShort((short)rRec.io().getType());
@@ -444,7 +444,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_PAGE_REMOVE:
                 RemoveRecord rmRec = (RemoveRecord)record;
 
-                buf.putInt(rmRec.cacheId());
+                buf.putInt(rmRec.groupId());
                 buf.putLong(rmRec.pageId());
 
                 buf.putShort((short)rmRec.index());
@@ -455,7 +455,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_PAGE_INNER_REPLACE:
                 InnerReplaceRecord<?> irRec = (InnerReplaceRecord<?>)record;
 
-                buf.putInt(irRec.cacheId());
+                buf.putInt(irRec.groupId());
                 buf.putLong(irRec.pageId());
 
                 buf.putShort((short)irRec.destinationIndex());
@@ -468,7 +468,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_FORWARD_PAGE_SPLIT:
                 SplitForwardPageRecord sfRec = (SplitForwardPageRecord)record;
 
-                buf.putInt(sfRec.cacheId());
+                buf.putInt(sfRec.groupId());
                 buf.putLong(sfRec.pageId());
 
                 buf.putLong(sfRec.forwardId());
@@ -483,7 +483,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_EXISTING_PAGE_SPLIT:
                 SplitExistingPageRecord seRec = (SplitExistingPageRecord)record;
 
-                buf.putInt(seRec.cacheId());
+                buf.putInt(seRec.groupId());
                 buf.putLong(seRec.pageId());
 
                 buf.putShort((short)seRec.middleIndex());
@@ -494,7 +494,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_PAGE_MERGE:
                 MergeRecord<?> mRec = (MergeRecord<?>)record;
 
-                buf.putInt(mRec.cacheId());
+                buf.putInt(mRec.groupId());
                 buf.putLong(mRec.pageId());
 
                 buf.putLong(mRec.parentId());
@@ -507,7 +507,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGES_LIST_SET_NEXT:
                 PagesListSetNextRecord plNextRec = (PagesListSetNextRecord)record;
 
-                buf.putInt(plNextRec.cacheId());
+                buf.putInt(plNextRec.groupId());
                 buf.putLong(plNextRec.pageId());
 
                 buf.putLong(plNextRec.nextPageId());
@@ -517,7 +517,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGES_LIST_SET_PREVIOUS:
                 PagesListSetPreviousRecord plPrevRec = (PagesListSetPreviousRecord)record;
 
-                buf.putInt(plPrevRec.cacheId());
+                buf.putInt(plPrevRec.groupId());
                 buf.putLong(plPrevRec.pageId());
 
                 buf.putLong(plPrevRec.previousPageId());
@@ -527,7 +527,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGES_LIST_INIT_NEW_PAGE:
                 PagesListInitNewPageRecord plNewRec = (PagesListInitNewPageRecord)record;
 
-                buf.putInt(plNewRec.cacheId());
+                buf.putInt(plNewRec.groupId());
                 buf.putLong(plNewRec.pageId());
                 buf.putInt(plNewRec.ioType());
                 buf.putInt(plNewRec.ioVersion());
@@ -541,7 +541,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGES_LIST_ADD_PAGE:
                 PagesListAddPageRecord plAddRec = (PagesListAddPageRecord)record;
 
-                buf.putInt(plAddRec.cacheId());
+                buf.putInt(plAddRec.groupId());
                 buf.putLong(plAddRec.pageId());
 
                 buf.putLong(plAddRec.dataPageId());
@@ -551,7 +551,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGES_LIST_REMOVE_PAGE:
                 PagesListRemovePageRecord plRmvRec = (PagesListRemovePageRecord)record;
 
-                buf.putInt(plRmvRec.cacheId());
+                buf.putInt(plRmvRec.groupId());
                 buf.putLong(plRmvRec.pageId());
 
                 buf.putLong(plRmvRec.removedPageId());
@@ -561,7 +561,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case BTREE_FIX_REMOVE_ID:
                 FixRemoveId frRec = (FixRemoveId)record;
 
-                buf.putInt(frRec.cacheId());
+                buf.putInt(frRec.groupId());
                 buf.putLong(frRec.pageId());
 
                 buf.putLong(frRec.removeId());
@@ -571,7 +571,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case TRACKING_PAGE_DELTA:
                 TrackingPageDeltaRecord tpDelta = (TrackingPageDeltaRecord)record;
 
-                buf.putInt(tpDelta.cacheId());
+                buf.putInt(tpDelta.groupId());
                 buf.putLong(tpDelta.pageId());
 
                 buf.putLong(tpDelta.pageIdToMark());
@@ -583,7 +583,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case META_PAGE_UPDATE_NEXT_SNAPSHOT_ID:
                 MetaPageUpdateNextSnapshotId mpUpdateNextSnapshotId = (MetaPageUpdateNextSnapshotId)record;
 
-                buf.putInt(mpUpdateNextSnapshotId.cacheId());
+                buf.putInt(mpUpdateNextSnapshotId.groupId());
                 buf.putLong(mpUpdateNextSnapshotId.pageId());
 
                 buf.putLong(mpUpdateNextSnapshotId.nextSnapshotId());
@@ -594,7 +594,7 @@ public class RecordV1Serializer implements RecordSerializer {
                 MetaPageUpdateLastSuccessfulFullSnapshotId mpUpdateLastSuccFullSnapshotId =
                     (MetaPageUpdateLastSuccessfulFullSnapshotId)record;
 
-                buf.putInt(mpUpdateLastSuccFullSnapshotId.cacheId());
+                buf.putInt(mpUpdateLastSuccFullSnapshotId.groupId());
                 buf.putLong(mpUpdateLastSuccFullSnapshotId.pageId());
 
                 buf.putLong(mpUpdateLastSuccFullSnapshotId.lastSuccessfulFullSnapshotId());
@@ -605,7 +605,7 @@ public class RecordV1Serializer implements RecordSerializer {
                 MetaPageUpdateLastSuccessfulSnapshotId mpUpdateLastSuccSnapshotId =
                     (MetaPageUpdateLastSuccessfulSnapshotId)record;
 
-                buf.putInt(mpUpdateLastSuccSnapshotId.cacheId());
+                buf.putInt(mpUpdateLastSuccSnapshotId.groupId());
                 buf.putLong(mpUpdateLastSuccSnapshotId.pageId());
 
                 buf.putLong(mpUpdateLastSuccSnapshotId.lastSuccessfulSnapshotId());
@@ -617,7 +617,7 @@ public class RecordV1Serializer implements RecordSerializer {
                 MetaPageUpdateLastAllocatedIndex mpUpdateLastAllocatedIdx =
                         (MetaPageUpdateLastAllocatedIndex) record;
 
-                buf.putInt(mpUpdateLastAllocatedIdx.cacheId());
+                buf.putInt(mpUpdateLastAllocatedIdx.groupId());
                 buf.putLong(mpUpdateLastAllocatedIdx.pageId());
 
                 buf.putInt(mpUpdateLastAllocatedIdx.lastAllocatedIndex());
@@ -627,7 +627,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PART_META_UPDATE_STATE:
                 PartitionMetaStateRecord partMetaStateRecord = (PartitionMetaStateRecord) record;
 
-                buf.putInt(partMetaStateRecord.cacheId());
+                buf.putInt(partMetaStateRecord.groupId());
 
                 buf.putInt(partMetaStateRecord.partitionId());
 
@@ -640,7 +640,7 @@ public class RecordV1Serializer implements RecordSerializer {
             case PAGE_LIST_META_RESET_COUNT_RECORD:
                 PageListMetaResetCountRecord pageListMetaResetCntRecord = (PageListMetaResetCountRecord) record;
 
-                buf.putInt(pageListMetaResetCntRecord.cacheId());
+                buf.putInt(pageListMetaResetCntRecord.groupId());
                 buf.putLong(pageListMetaResetCntRecord.pageId());
 
                 break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
index 02c521e..5bef372 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
@@ -64,10 +64,10 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             FullPageId fullId1 = allocatePage(mem);
             FullPageId fullId2 = allocatePage(mem);
 
-            long page1 = mem.acquirePage(fullId1.cacheId(), fullId1.pageId());
+            long page1 = mem.acquirePage(fullId1.groupId(), fullId1.pageId());
 
             try {
-                long page2 = mem.acquirePage(fullId2.cacheId(), fullId2.pageId());
+                long page2 = mem.acquirePage(fullId2.groupId(), fullId2.pageId());
 
                 info("Allocated pages [page1Id=" + fullId1.pageId() + ", page1=" + page1 +
                     ", page2Id=" + fullId2.pageId() + ", page2=" + page2 + ']');
@@ -84,11 +84,11 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                     readPage(mem, fullId2.pageId(), page2, 2);
                 }
                 finally {
-                    mem.releasePage(fullId2.cacheId(), fullId2.pageId(), page2);
+                    mem.releasePage(fullId2.groupId(), fullId2.pageId(), page2);
                 }
             }
             finally {
-                mem.releasePage(fullId1.cacheId(), fullId1.pageId(), page1);
+                mem.releasePage(fullId1.groupId(), fullId1.pageId(), page1);
             }
         }
         finally {
@@ -139,7 +139,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
 
                 pages.add(fullId);
 
-                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
                 try {
                     if (i % 64 == 0)
@@ -148,14 +148,14 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                     writePage(mem, fullId.pageId(), page, i + 1);
                 }
                 finally {
-                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                    mem.releasePage(fullId.groupId(), fullId.pageId(), page);
                 }
             }
 
             for (int i = 0; i < pagesCnt; i++) {
                 FullPageId fullId = pages.get(i);
 
-                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
                 try {
                     if (i % 64 == 0)
@@ -164,7 +164,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                     readPage(mem, fullId.pageId(), page, i + 1);
                 }
                 finally {
-                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                    mem.releasePage(fullId.groupId(), fullId.pageId(), page);
                 }
             }
         }
@@ -190,7 +190,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                 handles.add(allocatePage(mem));
 
             for (FullPageId fullId : handles)
-                mem.freePage(fullId.cacheId(), fullId.pageId());
+                mem.freePage(fullId.groupId(), fullId.pageId());
 
             for (int i = 0; i < pages; i++)
                 assertFalse(handles.add(allocatePage(mem)));
@@ -219,9 +219,9 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
 
             // Check that initial pages are accessible.
             for (FullPageId id : old) {
-                long pageApsPtr = mem.acquirePage(id.cacheId(), id.pageId());
+                long pageApsPtr = mem.acquirePage(id.groupId(), id.pageId());
                 try {
-                    long pageAddr = mem.writeLock(id.cacheId(), id.pageId(), pageApsPtr);
+                    long pageAddr = mem.writeLock(id.groupId(), id.pageId(), pageApsPtr);
 
                     assertNotNull(pageAddr);
 
@@ -230,47 +230,47 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
 
                         PageIO.setPageId(pageAddr, updId);
 
-                        updated.add(new FullPageId(updId, id.cacheId()));
+                        updated.add(new FullPageId(updId, id.groupId()));
                     }
                     finally {
-                        mem.writeUnlock(id.cacheId(), id.pageId(), pageApsPtr, null, true);
+                        mem.writeUnlock(id.groupId(), id.pageId(), pageApsPtr, null, true);
                     }
                 }
                 finally {
-                    mem.releasePage(id.cacheId(), id.pageId(), pageApsPtr);
+                    mem.releasePage(id.groupId(), id.pageId(), pageApsPtr);
                 }
             }
 
             // Check that updated pages are inaccessible using old IDs.
             for (FullPageId id : old) {
-                long pageApsPtr = mem.acquirePage(id.cacheId(), id.pageId());
+                long pageApsPtr = mem.acquirePage(id.groupId(), id.pageId());
                 try {
-                    long pageAddr = mem.writeLock(id.cacheId(), id.pageId(), pageApsPtr);
+                    long pageAddr = mem.writeLock(id.groupId(), id.pageId(), pageApsPtr);
 
                     if (pageAddr != 0L) {
-                        mem.writeUnlock(id.cacheId(), id.pageId(), pageApsPtr, null, false);
+                        mem.writeUnlock(id.groupId(), id.pageId(), pageApsPtr, null, false);
 
                         fail("Was able to acquire page write lock.");
                     }
 
-                    mem.readLock(id.cacheId(), id.pageId(), pageApsPtr);
+                    mem.readLock(id.groupId(), id.pageId(), pageApsPtr);
 
                     if (pageAddr != 0) {
-                        mem.readUnlock(id.cacheId(), id.pageId(), pageApsPtr);
+                        mem.readUnlock(id.groupId(), id.pageId(), pageApsPtr);
 
                         fail("Was able to acquire page read lock.");
                     }
                 }
                 finally {
-                    mem.releasePage(id.cacheId(), id.pageId(), pageApsPtr);
+                    mem.releasePage(id.groupId(), id.pageId(), pageApsPtr);
                 }
             }
 
             // Check that updated pages are accessible using new IDs.
             for (FullPageId id : updated) {
-                long pageApsPtr = mem.acquirePage(id.cacheId(), id.pageId());
+                long pageApsPtr = mem.acquirePage(id.groupId(), id.pageId());
                 try {
-                    long pageAddr = mem.writeLock(id.cacheId(), id.pageId(), pageApsPtr);
+                    long pageAddr = mem.writeLock(id.groupId(), id.pageId(), pageApsPtr);
 
                     assertNotSame(0L, pageAddr);
 
@@ -278,10 +278,10 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                         assertEquals(id.pageId(), PageIO.getPageId(pageAddr));
                     }
                     finally {
-                        mem.writeUnlock(id.cacheId(), id.pageId(), pageApsPtr, null, false);
+                        mem.writeUnlock(id.groupId(), id.pageId(), pageApsPtr, null, false);
                     }
 
-                    pageAddr = mem.readLock(id.cacheId(), id.pageId(), pageApsPtr);
+                    pageAddr = mem.readLock(id.groupId(), id.pageId(), pageApsPtr);
 
                     assertNotSame(0L, pageAddr);
 
@@ -289,11 +289,11 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                         assertEquals(id.pageId(), PageIO.getPageId(pageAddr));
                     }
                     finally {
-                        mem.readUnlock(id.cacheId(), id.pageId(), pageApsPtr);
+                        mem.readUnlock(id.groupId(), id.pageId(), pageApsPtr);
                     }
                 }
                 finally {
-                    mem.releasePage(id.cacheId(), id.pageId(), pageApsPtr);
+                    mem.releasePage(id.groupId(), id.pageId(), pageApsPtr);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
index e086258..c248c35 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
@@ -213,18 +213,18 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract
         PageMemory mem = shared.database().memoryPolicy(null).pageMemory();
 
         for (FullPageId fullId : pages) {
-            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
             try {
-                long pageAddr = mem.readLock(fullId.cacheId(), fullId.pageId(), page);
+                long pageAddr = mem.readLock(fullId.groupId(), fullId.pageId(), page);
 
                 for (int j = PageIO.COMMON_HEADER_END; j < mem.pageSize(); j += 4)
                     assertEquals(j + (int)fullId.pageId(), PageUtils.getInt(pageAddr, j));
 
-                mem.readUnlock(fullId.cacheId(), fullId.pageId(), page);
+                mem.readUnlock(fullId.groupId(), fullId.pageId(), page);
             }
             finally {
-                mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                mem.releasePage(fullId.groupId(), fullId.pageId(), page);
             }
         }
     }
@@ -252,10 +252,10 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract
         for (int i = 0; i < totalPages; i++) {
             FullPageId fullId = pages[i];
 
-            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
             try {
-                long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);
+                long pageAddr = mem.writeLock(fullId.groupId(), fullId.pageId(), page);
 
                 PageIO.setPageId(pageAddr, fullId.pageId());
 
@@ -264,11 +264,11 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract
                         PageUtils.putInt(pageAddr, j, j + (int)fullId.pageId());
                 }
                 finally {
-                    mem.writeUnlock(fullId.cacheId(), fullId.pageId(), page, null, true);
+                    mem.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, true);
                 }
             }
             finally {
-                mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                mem.releasePage(fullId.groupId(), fullId.pageId(), page);
             }
         }
 
@@ -340,17 +340,17 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract
         wal.fsync(wal.log(new CheckpointRecord(null)));
 
         for (FullPageId fullId : pages) {
-            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
             try {
                 assertFalse("Page has a temp heap copy after the last checkpoint: [cacheId=" +
-                    fullId.cacheId() + ", pageId=" + fullId.pageId() + "]", mem.hasTempCopy(page));
+                    fullId.groupId() + ", pageId=" + fullId.pageId() + "]", mem.hasTempCopy(page));
 
                 assertFalse("Page is dirty after the last checkpoint: [cacheId=" +
-                    fullId.cacheId() + ", pageId=" + fullId.pageId() + "]", mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                    fullId.groupId() + ", pageId=" + fullId.pageId() + "]", mem.isDirty(fullId.groupId(), fullId.pageId(), page));
             }
             finally {
-                mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                mem.releasePage(fullId.groupId(), fullId.pageId(), page);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
index e70c56e..297e69f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsCheckpointSimulationWithRealCpDisabledTest.java
@@ -228,9 +228,9 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
 
             // Check getForInitialWrite methods.
             for (FullPageId fullId : initWrites) {
-                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
                 try {
-                    long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);
+                    long pageAddr = mem.writeLock(fullId.groupId(), fullId.pageId(), page);
 
                     try {
                         DataPageIO.VERSIONS.latest().initNewPage(pageAddr, fullId.pageId(), mem.pageSize());
@@ -239,11 +239,11 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                             PageUtils.putByte(pageAddr, i, (byte)0xAB);
                     }
                     finally {
-                        mem.writeUnlock(fullId.cacheId(), fullId.pageId(), page, null, true);
+                        mem.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, true);
                     }
                 }
                 finally {
-                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                    mem.releasePage(fullId.groupId(), fullId.pageId(), page);
                 }
             }
 
@@ -527,26 +527,26 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                 pageIds[i] = new FullPageId(mem.allocatePage(cacheId, 0, PageIdAllocator.FLAG_DATA), cacheId);
 
             for (FullPageId fullId : pageIds) {
-                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
                 try {
-                    assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page)); //page is dirty right after allocation
+                    assertTrue(mem.isDirty(fullId.groupId(), fullId.pageId(), page)); //page is dirty right after allocation
 
-                    long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);
+                    long pageAddr = mem.writeLock(fullId.groupId(), fullId.pageId(), page);
 
                     PageIO.setPageId(pageAddr, fullId.pageId());
 
                     try {
-                        assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                        assertTrue(mem.isDirty(fullId.groupId(), fullId.pageId(), page));
                     }
                     finally {
-                        mem.writeUnlock(fullId.cacheId(), fullId.pageId(),page, null,true);
+                        mem.writeUnlock(fullId.groupId(), fullId.pageId(),page, null,true);
                     }
 
-                    assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                    assertTrue(mem.isDirty(fullId.groupId(), fullId.pageId(), page));
                 }
                 finally {
-                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                    mem.releasePage(fullId.groupId(), fullId.pageId(), page);
                 }
             }
         }
@@ -564,24 +564,24 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
 
                 ByteBuffer buf = ByteBuffer.allocate(mem.pageSize());
 
-                long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+                long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
                 try {
-                    assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                    assertTrue(mem.isDirty(fullId.groupId(), fullId.pageId(), page));
 
-                    long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);
+                    long pageAddr = mem.writeLock(fullId.groupId(), fullId.pageId(), page);
 
                     try {
-                        assertFalse(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                        assertFalse(mem.isDirty(fullId.groupId(), fullId.pageId(), page));
 
                         for (int i = PageIO.COMMON_HEADER_END; i < mem.pageSize(); i++)
                             PageUtils.putByte(pageAddr, i, (byte)1);
                     }
                     finally {
-                        mem.writeUnlock(fullId.cacheId(), fullId.pageId(), page, null, true);
+                        mem.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, true);
                     }
 
-                    assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                    assertTrue(mem.isDirty(fullId.groupId(), fullId.pageId(), page));
 
                     buf.rewind();
 
@@ -593,7 +593,7 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                         assertEquals((byte)0, buf.get());
                 }
                 finally {
-                    mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                    mem.releasePage(fullId.groupId(), fullId.pageId(), page);
                 }
             }
         }
@@ -604,12 +604,12 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
         mem.finishCheckpoint();
 
         for (FullPageId fullId : pageIds) {
-            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
             try {
-                assertTrue(mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                assertTrue(mem.isDirty(fullId.groupId(), fullId.pageId(), page));
             }
             finally {
-                mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                mem.releasePage(fullId.groupId(), fullId.pageId(), page);
             }
         }
     }
@@ -618,9 +618,9 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
      * @throws Exception if failed.
      */
     private void writePageData(FullPageId fullId, PageMemory mem) throws Exception {
-        long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+        long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
         try {
-            long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);
+            long pageAddr = mem.writeLock(fullId.groupId(), fullId.pageId(), page);
 
             try {
                 DataPageIO.VERSIONS.latest().initNewPage(pageAddr, fullId.pageId(), mem.pageSize());
@@ -631,11 +631,11 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                     PageUtils.putByte(pageAddr, i, (byte)rnd.nextInt(255));
             }
             finally {
-                mem.writeUnlock(fullId.cacheId(), fullId.pageId(), page, null, true);
+                mem.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, true);
             }
         }
         finally {
-            mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+            mem.releasePage(fullId.groupId(), fullId.pageId(), page);
         }
     }
 
@@ -693,9 +693,9 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
 
             assertNotNull("Missing WAL record for a written page: " + fullId, walData);
 
-            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
             try {
-                long pageAddr = mem.readLock(fullId.cacheId(), fullId.pageId(), page);
+                long pageAddr = mem.readLock(fullId.groupId(), fullId.pageId(), page);
 
                 try {
                     for (int i = PageIO.COMMON_HEADER_END; i < mem.pageSize(); i++) {
@@ -707,11 +707,11 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                     }
                 }
                 finally {
-                    mem.readUnlock(fullId.cacheId(), fullId.pageId(), page);
+                    mem.readUnlock(fullId.groupId(), fullId.pageId(), page);
                 }
             }
             finally {
-                mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                mem.releasePage(fullId.groupId(), fullId.pageId(), page);
             }
         }
     }
@@ -772,10 +772,10 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                         ig.context().cache().context().database().checkpointReadLock();
 
                         try {
-                            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+                            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
                             try {
-                                long pageAddr = mem.writeLock(fullId.cacheId(), fullId.pageId(), page);
+                                long pageAddr = mem.writeLock(fullId.groupId(), fullId.pageId(), page);
 
                                 PageIO.setPageId(pageAddr, fullId.pageId());
 
@@ -813,11 +813,11 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
                                     resMap.put(fullId, state);
                                 }
                                 finally {
-                                    mem.writeUnlock(fullId.cacheId(), fullId.pageId(),page, null,true);
+                                    mem.writeUnlock(fullId.groupId(), fullId.pageId(),page, null,true);
                                 }
                             }
                             finally {
-                                mem.releasePage(fullId.cacheId(), fullId.pageId(),page);}
+                                mem.releasePage(fullId.groupId(), fullId.pageId(),page);}
                             }
                             finally {
                                 ig.context().cache().context().database().checkpointReadUnlock();
@@ -958,17 +958,17 @@ public class IgnitePdsCheckpointSimulationWithRealCpDisabledTest extends GridCom
 
         for (FullPageId fullId : pages) {
 
-            long page = mem.acquirePage(fullId.cacheId(), fullId.pageId());
+            long page = mem.acquirePage(fullId.groupId(), fullId.pageId());
 
             try {
                 assertFalse("Page has a temp heap copy after the last checkpoint: [cacheId=" +
-                    fullId.cacheId() + ", pageId=" + fullId.pageId() + "]", mem.hasTempCopy(page));
+                    fullId.groupId() + ", pageId=" + fullId.pageId() + "]", mem.hasTempCopy(page));
 
                 assertFalse("Page is dirty after the last checkpoint: [cacheId=" +
-                    fullId.cacheId() + ", pageId=" + fullId.pageId() + "]", mem.isDirty(fullId.cacheId(), fullId.pageId(), page));
+                    fullId.groupId() + ", pageId=" + fullId.pageId() + "]", mem.isDirty(fullId.groupId(), fullId.pageId(), page));
             }
             finally {
-                mem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                mem.releasePage(fullId.groupId(), fullId.pageId(), page);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
index ea4a0e9..d7790c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/file/IgnitePdsEvictionTest.java
@@ -203,10 +203,10 @@ public class IgnitePdsEvictionTest extends GridCommonAbstractTest {
                     try {
                         FullPageId fullId = pageIds.get(i);
 
-                        long page = memory.acquirePage(fullId.cacheId(), fullId.pageId());
+                        long page = memory.acquirePage(fullId.groupId(), fullId.pageId());
 
                         try {
-                            final long pageAddr = memory.writeLock(fullId.cacheId(), fullId.pageId(), page);
+                            final long pageAddr = memory.writeLock(fullId.groupId(), fullId.pageId(), page);
 
                             try {
                                 PageIO.setPageId(pageAddr, fullId.pageId());
@@ -214,11 +214,11 @@ public class IgnitePdsEvictionTest extends GridCommonAbstractTest {
                                 PageUtils.putLong(pageAddr, PageIO.COMMON_HEADER_END, i * 2);
                             }
                             finally {
-                                memory.writeUnlock(fullId.cacheId(), fullId.pageId(), page, null, true);
+                                memory.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, true);
                             }
                         }
                         finally {
-                            memory.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                            memory.releasePage(fullId.groupId(), fullId.pageId(), page);
                         }
                     }
                     finally {
@@ -252,19 +252,19 @@ public class IgnitePdsEvictionTest extends GridCommonAbstractTest {
                     try {
                         final FullPageId fullId = pageIds.get(i);
 
-                        long page = memory.acquirePage(fullId.cacheId(), fullId.pageId());
+                        long page = memory.acquirePage(fullId.groupId(), fullId.pageId());
                         try {
-                            final long pageAddr = memory.readLock(fullId.cacheId(), fullId.pageId(), page);
+                            final long pageAddr = memory.readLock(fullId.groupId(), fullId.pageId(), page);
 
                             try {
                                 assertEquals(i * 2, PageUtils.getLong(pageAddr, PageIO.COMMON_HEADER_END));
                             }
                             finally {
-                                memory.readUnlock(fullId.cacheId(), fullId.pageId(), page);
+                                memory.readUnlock(fullId.groupId(), fullId.pageId(), page);
                             }
                         }
                         finally {
-                            memory.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                            memory.releasePage(fullId.groupId(), fullId.pageId(), page);
                         }
                     }
                     finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index 843fb5b..c5d6a8b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -934,7 +934,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
                     else if (rec instanceof PageDeltaRecord) {
                         PageDeltaRecord delta = (PageDeltaRecord)rec;
 
-                        FullPageId fullId = new FullPageId(delta.pageId(), delta.cacheId());
+                        FullPageId fullId = new FullPageId(delta.pageId(), delta.groupId());
 
                         byte[] pageData = rolledPages.get(fullId);
 
@@ -976,10 +976,10 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
                 ignite0.context().cache().context().database().checkpointReadLock();
 
                 try {
-                    long page = pageMem.acquirePage(fullId.cacheId(), fullId.pageId(), true);
+                    long page = pageMem.acquirePage(fullId.groupId(), fullId.pageId(), true);
 
                     try {
-                        long buf = pageMem.writeLock(fullId.cacheId(), fullId.pageId(), page, true);
+                        long buf = pageMem.writeLock(fullId.groupId(), fullId.pageId(), page, true);
 
                         try {
                             byte[] data = entry.getValue();
@@ -992,11 +992,11 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
                             }
                         }
                         finally {
-                            pageMem.writeUnlock(fullId.cacheId(), fullId.pageId(), page, null, false, true);
+                            pageMem.writeUnlock(fullId.groupId(), fullId.pageId(), page, null, false, true);
                         }
                     }
                     finally {
-                        pageMem.releasePage(fullId.cacheId(), fullId.pageId(), page);
+                        pageMem.releasePage(fullId.groupId(), fullId.pageId(), page);
                     }
                 }
                 finally {


[20/41] ignite git commit: IGNITE-5554 ServiceProcessor may process failed reassignments in timeout thread

Posted by sb...@apache.org.
IGNITE-5554 ServiceProcessor may process failed reassignments in timeout thread


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

Branch: refs/heads/ignite-5578-1
Commit: 517a23d252078e62a8addca55ca78497e5226b5b
Parents: 3314a45
Author: mcherkasov <mc...@gridgain.com>
Authored: Fri Jun 30 20:23:55 2017 +0300
Committer: mcherkasov <mc...@gridgain.com>
Committed: Fri Jul 7 17:44:16 2017 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           | 24 +++++++++-----------
 1 file changed, 11 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/517a23d2/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index d67f2d1..23a29f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -26,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
@@ -1509,18 +1510,11 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
                 }
 
                 @Override public void onTimeout() {
-                    GridSpinBusyLock busyLock = GridServiceProcessor.this.busyLock;
-
-                    if (busyLock == null || !busyLock.enterBusy())
-                        return;
-
-                    try {
-                        // Try again.
-                        onDeployment(dep, topVer);
-                    }
-                    finally {
-                        busyLock.leaveBusy();
-                    }
+                    depExe.execute(new DepRunnable() {
+                        @Override public void run0() {
+                            onDeployment(dep, topVer);
+                        }
+                    });
                 }
             });
         }
@@ -1716,7 +1710,11 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
                         }
 
                         @Override public void onTimeout() {
-                            onReassignmentFailed(topVer, retries);
+                            depExe.execute(new Runnable() {
+                                public void run() {
+                                    onReassignmentFailed(topVer, retries);
+                                }
+                            });
                         }
                     });
                 }


[40/41] ignite git commit: ignite-5075 Rename cacheId -> groupId

Posted by sb...@apache.org.
ignite-5075 Rename cacheId -> groupId


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

Branch: refs/heads/ignite-5578-1
Commit: f2568b76417aebc0e3c20003a8120b2d2adab3eb
Parents: 5686810
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 14:41:04 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 14:41:04 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/FullPageId.java     | 32 ++++-----
 .../ignite/internal/pagemem/PageSupport.java    | 36 +++++-----
 .../pagemem/store/IgnitePageStoreManager.java   | 40 +++++------
 .../delta/DataPageInsertFragmentRecord.java     |  6 +-
 .../wal/record/delta/DataPageInsertRecord.java  |  6 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |  6 +-
 .../delta/DataPageSetFreeListPageRecord.java    |  6 +-
 .../wal/record/delta/DataPageUpdateRecord.java  |  6 +-
 .../wal/record/delta/FixCountRecord.java        |  6 +-
 .../record/delta/FixLeftmostChildRecord.java    |  6 +-
 .../pagemem/wal/record/delta/FixRemoveId.java   |  6 +-
 .../wal/record/delta/InitNewPageRecord.java     |  6 +-
 .../wal/record/delta/InnerReplaceRecord.java    |  6 +-
 .../pagemem/wal/record/delta/InsertRecord.java  |  6 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |  6 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |  6 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |  6 +-
 .../wal/record/delta/MetaPageInitRecord.java    |  6 +-
 .../delta/MetaPageInitRootInlineRecord.java     |  6 +-
 .../record/delta/MetaPageInitRootRecord.java    |  4 +-
 .../delta/MetaPageUpdateLastAllocatedIndex.java |  4 +-
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |  4 +-
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |  4 +-
 .../delta/MetaPageUpdateNextSnapshotId.java     |  4 +-
 .../MetaPageUpdatePartitionDataRecord.java      |  6 +-
 .../wal/record/delta/NewRootInitRecord.java     |  6 +-
 .../wal/record/delta/PageDeltaRecord.java       | 14 ++--
 .../delta/PageListMetaResetCountRecord.java     |  6 +-
 .../record/delta/PagesListAddPageRecord.java    |  6 +-
 .../delta/PagesListInitNewPageRecord.java       |  6 +-
 .../record/delta/PagesListRemovePageRecord.java |  2 +-
 .../record/delta/PagesListSetNextRecord.java    |  6 +-
 .../delta/PagesListSetPreviousRecord.java       |  6 +-
 .../record/delta/PartitionDestroyRecord.java    | 20 +++---
 .../record/delta/PartitionMetaStateRecord.java  | 16 ++---
 .../pagemem/wal/record/delta/RecycleRecord.java |  6 +-
 .../pagemem/wal/record/delta/RemoveRecord.java  |  6 +-
 .../pagemem/wal/record/delta/ReplaceRecord.java |  6 +-
 .../record/delta/SplitExistingPageRecord.java   |  6 +-
 .../record/delta/SplitForwardPageRecord.java    |  6 +-
 .../record/delta/TrackingPageDeltaRecord.java   |  6 +-
 .../cache/persistence/DataStructure.java        | 46 ++++++------
 .../GridCacheDatabaseSharedManager.java         | 38 +++++-----
 .../persistence/GridCacheOffheapManager.java    |  4 +-
 .../cache/persistence/MetadataStorage.java      |  2 +-
 .../persistence/file/FilePageStoreManager.java  | 34 ++++-----
 .../persistence/freelist/FreeListImpl.java      |  6 +-
 .../cache/persistence/freelist/PagesList.java   | 36 +++++-----
 .../persistence/pagemem/FullPageIdTable.java    | 54 +++++++-------
 .../cache/persistence/pagemem/PageMemoryEx.java |  4 +-
 .../persistence/pagemem/PageMemoryImpl.java     | 72 +++++++++----------
 .../cache/persistence/tree/BPlusTree.java       | 16 ++---
 .../wal/serializer/RecordV1Serializer.java      | 76 ++++++++++----------
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  | 52 +++++++-------
 ...gnitePdsRecoveryAfterFileCorruptionTest.java | 24 +++----
 ...ckpointSimulationWithRealCpDisabledTest.java | 74 +++++++++----------
 .../db/file/IgnitePdsEvictionTest.java          | 16 ++---
 .../db/wal/IgniteWalRecoveryTest.java           | 10 +--
 .../pagemem/NoOpPageStoreManager.java           | 22 +++---
 59 files changed, 470 insertions(+), 470 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
index 20677ea..00f52c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
@@ -56,28 +56,28 @@ public class FullPageId {
     private final long effectivePageId;
 
     /** */
-    private final int cacheId;
+    private final int grpId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @return Hash code.
      */
-    public static int hashCode(int cacheId, long pageId) {
+    public static int hashCode(int grpId, long pageId) {
         long effectiveId = PageIdUtils.effectivePageId(pageId);
 
-        return U.hash(hashCode0(cacheId, effectiveId));
+        return U.hash(hashCode0(grpId, effectiveId));
     }
 
     /**
      * Will not clear link bits.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param effectivePageId Effective page ID.
      * @return Hash code.
      */
-    private static int hashCode0(int cacheId, long effectivePageId) {
-        return (int)(mix64(effectivePageId) ^ mix32(cacheId));
+    private static int hashCode0(int grpId, long effectivePageId) {
+        return (int)(mix64(effectivePageId) ^ mix32(grpId));
     }
 
     /**
@@ -107,11 +107,11 @@ public class FullPageId {
     }
     /**
      * @param pageId Page ID.
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      */
-    public FullPageId(long pageId, int cacheId) {
+    public FullPageId(long pageId, int grpId) {
         this.pageId = pageId;
-        this.cacheId = cacheId;
+        this.grpId = grpId;
 
         effectivePageId = PageIdUtils.effectivePageId(pageId);
     }
@@ -124,10 +124,10 @@ public class FullPageId {
     }
 
     /**
-     * @return Cache ID.
+     * @return Cache group ID.
      */
-    public int cacheId() {
-        return cacheId;
+    public int groupId() {
+        return grpId;
     }
 
     /** {@inheritDoc} */
@@ -140,18 +140,18 @@ public class FullPageId {
 
         FullPageId that = (FullPageId)o;
 
-        return effectivePageId == that.effectivePageId && cacheId == that.cacheId;
+        return effectivePageId == that.effectivePageId && grpId == that.grpId;
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return hashCode0(cacheId, effectivePageId);
+        return hashCode0(grpId, effectivePageId);
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return new SB("FullPageId [pageId=").appendHex(pageId)
             .a(", effectivePageId=").appendHex(effectivePageId)
-            .a(", cacheId=").a(cacheId).a(']').toString();
+            .a(", grpId=").a(grpId).a(']').toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
index f4b2d96..ed2311f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
@@ -28,87 +28,87 @@ public interface PageSupport {
      * released by calling {@link #releasePage(int, long, long)}. This method will allocate page with given ID if it doesn't
      * exist.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @return Page pointer.
      * @throws IgniteCheckedException If failed.
      */
-    public long acquirePage(int cacheId, long pageId) throws IgniteCheckedException;
+    public long acquirePage(int grpId, long pageId) throws IgniteCheckedException;
 
     /**
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID to release.
      * @param page Page pointer.
      */
-    public void releasePage(int cacheId, long pageId, long page);
+    public void releasePage(int grpId, long pageId, long page);
 
     /**
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      * @return Pointer for reading the page.
      */
-    public long readLock(int cacheId, long pageId, long page);
+    public long readLock(int grpId, long pageId, long page);
 
     /**
      * Obtains read lock without checking page tag.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      * @return Pointer for reading the page.
      */
-    public long readLockForce(int cacheId, long pageId, long page);
+    public long readLockForce(int grpId, long pageId, long page);
 
     /**
      * Releases locked page.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      */
-    public void readUnlock(int cacheId, long pageId, long page);
+    public void readUnlock(int grpId, long pageId, long page);
 
     /**
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      * @return Address of a buffer with contents of the given page or
      *            {@code 0L} if attempt to take the write lock failed.
      */
-    public long writeLock(int cacheId, long pageId, long page);
+    public long writeLock(int grpId, long pageId, long page);
 
     /**
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      * @return Address of a buffer with contents of the given page or
      *            {@code 0L} if attempt to take the write lock failed.
      */
-    public long tryWriteLock(int cacheId, long pageId, long page);
+    public long tryWriteLock(int grpId, long pageId, long page);
 
     /**
      * Releases locked page.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      * @param walPlc {@code True} if page should be recorded to WAL, {@code false} if the page must not
      *      be recorded and {@code null} for the default behavior.
      * @param dirtyFlag Determines whether the page was modified since the last checkpoint.
      */
-    public void writeUnlock(int cacheId, long pageId, long page, Boolean walPlc,
+    public void writeUnlock(int grpId, long pageId, long page, Boolean walPlc,
         boolean dirtyFlag);
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param page Page pointer.
      * @return {@code True} if the page is dirty.
      */
-    public boolean isDirty(int cacheId, long pageId, long page);
+    public boolean isDirty(int grpId, long pageId, long page);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index fa6e9e4..a1b766f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -49,8 +49,8 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
      * @param cacheData Cache data of the cache being started.
      * @throws IgniteCheckedException If failed to handle cache start callback.
      */
-    public void initializeForCache(CacheGroupDescriptor grpDesc,
-        StoredCacheData cacheData) throws IgniteCheckedException;
+    public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cacheData)
+        throws IgniteCheckedException;
 
     /**
      * Callback called when a cache is stopping. After this callback is invoked, no data associated with
@@ -84,12 +84,12 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
     /**
      * Reads a page for the given cache ID. Cache ID may be {@code 0} if the page is a meta page.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId PageID to read.
      * @param pageBuf Page buffer to write to.
      * @throws IgniteCheckedException If failed to read the page.
      */
-    public void read(int cacheId, long pageId, ByteBuffer pageBuf) throws IgniteCheckedException;
+    public void read(int grpId, long pageId, ByteBuffer pageBuf) throws IgniteCheckedException;
 
     /**
      * Checks if partition store exists.
@@ -104,77 +104,77 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
     /**
      * Reads a header of a page store.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param partId Partition ID.
      * @param buf Buffer to write to.
      * @throws IgniteCheckedException If failed.
      */
-    public void readHeader(int cacheId, int partId, ByteBuffer buf) throws IgniteCheckedException;
+    public void readHeader(int grpId, int partId, ByteBuffer buf) throws IgniteCheckedException;
 
     /**
      * Writes the page for the given cache ID. Cache ID may be {@code 0} if the page is a meta page.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param pageBuf Page buffer to write.
      * @throws IgniteCheckedException If failed to write page.
      */
-    public void write(int cacheId, long pageId, ByteBuffer pageBuf, int tag) throws IgniteCheckedException;
+    public void write(int grpId, long pageId, ByteBuffer pageBuf, int tag) throws IgniteCheckedException;
 
     /**
      * Gets page offset within the page store file.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @return Page offset.
      * @throws IgniteCheckedException If failed.
      */
-    public long pageOffset(int cacheId, long pageId) throws IgniteCheckedException;
+    public long pageOffset(int grpId, long pageId) throws IgniteCheckedException;
 
     /**
      * Makes sure that all previous writes to the store has been written to disk.
      *
-     * @param cacheId Cache ID to sync.
+     * @param grpId Cache group ID to sync.
      * @param partId Partition ID to sync.
      * @throws IgniteCheckedException If IO error occurred while running sync.
      */
-    public void sync(int cacheId, int partId) throws IgniteCheckedException;
+    public void sync(int grpId, int partId) throws IgniteCheckedException;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param partId Partition ID.
      * @throws IgniteCheckedException If failed.
      */
-    public void ensure(int cacheId, int partId) throws IgniteCheckedException;
+    public void ensure(int grpId, int partId) throws IgniteCheckedException;
 
     /**
      * Allocates a page for the given page space.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param partId Partition ID. Used only if {@code flags} is equal to {@link PageMemory#FLAG_DATA}.
      * @param flags Page allocation flags.
      * @return Allocated page ID.
      * @throws IgniteCheckedException If IO exception occurred while allocating a page ID.
      */
-    public long allocatePage(int cacheId, int partId, byte flags) throws IgniteCheckedException;
+    public long allocatePage(int grpId, int partId, byte flags) throws IgniteCheckedException;
 
     /**
      * Gets total number of allocated pages for the given space.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param partId Partition ID.
      * @return Number of allocated pages.
      * @throws IgniteCheckedException If failed.
      */
-    public int pages(int cacheId, int partId) throws IgniteCheckedException;
+    public int pages(int grpId, int partId) throws IgniteCheckedException;
 
     /**
      * Gets meta page ID for specified cache.
      *
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @return Meta page ID.
      */
-    public long metaPageId(int cacheId);
+    public long metaPageId(int grpId);
 
     /**
      * @return Saved cache configurations.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
index 042fbe4..3b2ced1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
@@ -32,18 +32,18 @@ public class DataPageInsertFragmentRecord extends PageDeltaRecord {
     private final byte[] payload;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param payload Fragment payload.
      * @param lastLink Link to the last entry fragment.
      */
     public DataPageInsertFragmentRecord(
-        final int cacheId,
+        final int grpId,
         final long pageId,
         final byte[] payload,
         final long lastLink
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         this.lastLink = lastLink;
         this.payload = payload;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
index 17425fe..e480cc8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
@@ -29,16 +29,16 @@ public class DataPageInsertRecord extends PageDeltaRecord {
     private byte[] payload;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param payload Remainder of the record.
      */
     public DataPageInsertRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         byte[] payload
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         this.payload = payload;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
index 3aabe08..b80600c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
@@ -30,12 +30,12 @@ public class DataPageRemoveRecord extends PageDeltaRecord {
     private int itemId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param itemId Item ID.
      */
-    public DataPageRemoveRecord(int cacheId, long pageId, int itemId) {
-        super(cacheId, pageId);
+    public DataPageRemoveRecord(int grpId, long pageId, int itemId) {
+        super(grpId, pageId);
 
         this.itemId = itemId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
index 82281de..a7fd31f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
@@ -31,12 +31,12 @@ public class DataPageSetFreeListPageRecord extends PageDeltaRecord {
     private long freeListPage;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param freeListPage Free list page ID.
      */
-    public DataPageSetFreeListPageRecord(int cacheId, long pageId, long freeListPage) {
-        super(cacheId, pageId);
+    public DataPageSetFreeListPageRecord(int grpId, long pageId, long freeListPage) {
+        super(grpId, pageId);
 
         this.freeListPage = freeListPage;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
index d4e2811..6207f41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
@@ -32,18 +32,18 @@ public class DataPageUpdateRecord extends PageDeltaRecord {
     private byte[] payload;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param itemId Item ID.
      * @param payload Record data.
      */
     public DataPageUpdateRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         int itemId,
         byte[] payload
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         this.payload = payload;
         this.itemId = itemId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
index 3ab1c4a..80e06ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
@@ -30,11 +30,11 @@ public class FixCountRecord extends PageDeltaRecord {
     private int cnt;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      */
-    public FixCountRecord(int cacheId, long pageId, int cnt) {
-        super(cacheId, pageId);
+    public FixCountRecord(int grpId, long pageId, int cnt) {
+        super(grpId, pageId);
 
         this.cnt = cnt;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
index 15ee7dd..a34e8e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
@@ -30,12 +30,12 @@ public class FixLeftmostChildRecord extends PageDeltaRecord {
     private long rightId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param rightId Right ID.
      */
-    public FixLeftmostChildRecord(int cacheId, long pageId, long rightId) {
-        super(cacheId, pageId);
+    public FixLeftmostChildRecord(int grpId, long pageId, long rightId) {
+        super(grpId, pageId);
 
         this.rightId = rightId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
index b6ea410..1ec845d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
@@ -30,12 +30,12 @@ public class FixRemoveId extends PageDeltaRecord {
     private long rmvId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param rmvId Remove ID.
      */
-    public FixRemoveId(int cacheId, long pageId, long rmvId) {
-        super(cacheId, pageId);
+    public FixRemoveId(int grpId, long pageId, long rmvId) {
+        super(grpId, pageId);
 
         this.rmvId = rmvId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
index f883e06..15bd2da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
@@ -39,14 +39,14 @@ public class InitNewPageRecord extends PageDeltaRecord {
     protected long newPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param ioType IO type.
      * @param ioVer IO version.
      * @param newPageId New page ID.
      */
-    public InitNewPageRecord(int cacheId, long pageId, int ioType, int ioVer, long newPageId) {
-        super(cacheId, pageId);
+    public InitNewPageRecord(int grpId, long pageId, int ioType, int ioVer, long newPageId) {
+        super(grpId, pageId);
 
         this.ioType = ioType;
         this.ioVer = ioVer;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
index 35d23c1..6e6879d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
@@ -38,14 +38,14 @@ public class InnerReplaceRecord<L> extends PageDeltaRecord {
     private long rmvId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param dstIdx Destination index.
      * @param srcPageId Source page ID.
      * @param srcIdx Source index.
      */
-    public InnerReplaceRecord(int cacheId, long pageId, int dstIdx, long srcPageId, int srcIdx, long rmvId) {
-        super(cacheId, pageId);
+    public InnerReplaceRecord(int grpId, long pageId, int dstIdx, long srcPageId, int srcIdx, long rmvId) {
+        super(grpId, pageId);
 
         this.dstIdx = dstIdx;
         this.srcPageId = srcPageId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
index d0e66a3..f6ebe87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
@@ -42,7 +42,7 @@ public class InsertRecord<L> extends PageDeltaRecord {
     private BPlusIO<L> io;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param io IO.
      * @param idx Index.
@@ -50,14 +50,14 @@ public class InsertRecord<L> extends PageDeltaRecord {
      * @param rightId Right ID.
      */
     public InsertRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         BPlusIO<L> io,
         int idx,
         byte[] rowBytes,
         long rightId
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         this.io = io;
         this.idx = idx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
index 84770d5..2d481d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
@@ -44,15 +44,15 @@ public class MergeRecord<L> extends PageDeltaRecord {
     private boolean emptyBranch;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param prntId Parent ID.
      * @param prntIdx Index in parent page.
      * @param rightId Right ID.
      * @param emptyBranch We are merging empty branch.
      */
-    public MergeRecord(int cacheId, long pageId, long prntId, int prntIdx, long rightId, boolean emptyBranch) {
-        super(cacheId, pageId);
+    public MergeRecord(int grpId, long pageId, long prntId, int prntIdx, long rightId, boolean emptyBranch) {
+        super(grpId, pageId);
 
         this.prntId = prntId;
         this.rightId = rightId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index 1f1e976..ce9e06b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -29,12 +29,12 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     private long rootId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache ID.
      * @param pageId  Page ID.
      * @param rootId Root ID.
      */
-    public MetaPageAddRootRecord(int cacheId, long pageId, long rootId) {
-        super(cacheId, pageId);
+    public MetaPageAddRootRecord(int grpId, long pageId, long rootId) {
+        super(grpId, pageId);
 
         this.rootId = rootId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index 3fc42a1..1e93ab9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -26,11 +26,11 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMeta
  */
 public class MetaPageCutRootRecord extends PageDeltaRecord {
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      */
-    public MetaPageCutRootRecord(int cacheId, long pageId) {
-        super(cacheId, pageId);
+    public MetaPageCutRootRecord(int grpId, long pageId) {
+        super(grpId, pageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
index 692b8f1..b1e701a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
@@ -36,14 +36,14 @@ public class MetaPageInitRecord extends InitNewPageRecord {
     private int ioType;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param ioType IO type.
      * @param treeRoot Tree root.
      * @param reuseListRoot Reuse list root.
      */
-    public MetaPageInitRecord(int cacheId, long pageId, int ioType, int ioVer, long treeRoot, long reuseListRoot) {
-        super(cacheId, pageId, ioType, ioVer, pageId);
+    public MetaPageInitRecord(int grpId, long pageId, int ioType, int ioVer, long treeRoot, long reuseListRoot) {
+        super(grpId, pageId, ioType, ioVer, pageId);
 
         assert ioType == PageIO.T_META || ioType == PageIO.T_PART_META;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineRecord.java
index 7eb8426..2b82444 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootInlineRecord.java
@@ -30,13 +30,13 @@ public class MetaPageInitRootInlineRecord extends MetaPageInitRootRecord {
     private final int inlineSize;
 
     /**
-     * @param cacheId
+     * @param grpId Cache group ID.
      * @param pageId Meta page ID.
      * @param rootId
      * @param inlineSize Inline size.
      */
-    public MetaPageInitRootInlineRecord(int cacheId, long pageId, long rootId, int inlineSize) {
-        super(cacheId, pageId, rootId);
+    public MetaPageInitRootInlineRecord(int grpId, long pageId, long rootId, int inlineSize) {
+        super(grpId, pageId, rootId);
         this.inlineSize = inlineSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index 0eb61b5..ebdf3d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -31,8 +31,8 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
     /**
      * @param pageId Meta page ID.
      */
-    public MetaPageInitRootRecord(int cacheId, long pageId, long rootId) {
-        super(cacheId, pageId);
+    public MetaPageInitRootRecord(int grpId, long pageId, long rootId) {
+        super(grpId, pageId);
 
         assert pageId != rootId;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
index 548735e..60aebde 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
@@ -32,8 +32,8 @@ public class MetaPageUpdateLastAllocatedIndex extends PageDeltaRecord {
     /**
      * @param pageId Meta page ID.
      */
-    public MetaPageUpdateLastAllocatedIndex(int cacheId, long pageId, int lastAllocatedIdx) {
-        super(cacheId, pageId);
+    public MetaPageUpdateLastAllocatedIndex(int grpId, long pageId, int lastAllocatedIdx) {
+        super(grpId, pageId);
 
         this.lastAllocatedIdx = lastAllocatedIdx;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
index 34be353..03f945f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
@@ -31,8 +31,8 @@ public class MetaPageUpdateLastSuccessfulFullSnapshotId extends PageDeltaRecord
     /**
      * @param pageId Meta page ID.
      */
-    public MetaPageUpdateLastSuccessfulFullSnapshotId(int cacheId, long pageId, long lastSuccessfulFullSnapshotId) {
-        super(cacheId, pageId);
+    public MetaPageUpdateLastSuccessfulFullSnapshotId(int grpId, long pageId, long lastSuccessfulFullSnapshotId) {
+        super(grpId, pageId);
 
         this.lastSuccessfulFullSnapshotId = lastSuccessfulFullSnapshotId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
index 7fca5dc..07798bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
@@ -34,8 +34,8 @@ public class MetaPageUpdateLastSuccessfulSnapshotId extends PageDeltaRecord {
      * @param pageId Meta page ID.
      * @param snapshotTag
      */
-    public MetaPageUpdateLastSuccessfulSnapshotId(int cacheId, long pageId, long lastSuccessfulSnapshotId, long snapshotTag) {
-        super(cacheId, pageId);
+    public MetaPageUpdateLastSuccessfulSnapshotId(int grpId, long pageId, long lastSuccessfulSnapshotId, long snapshotTag) {
+        super(grpId, pageId);
 
         this.lastSuccessfulSnapshotId = lastSuccessfulSnapshotId;
         this.lastSuccessfulSnapshotTag = snapshotTag;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
index 0cd72ec..e2d1267 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
@@ -31,8 +31,8 @@ public class MetaPageUpdateNextSnapshotId extends PageDeltaRecord {
     /**
      * @param pageId Meta page ID.
      */
-    public MetaPageUpdateNextSnapshotId(int cacheId, long pageId, long nextSnapshotId) {
-        super(cacheId, pageId);
+    public MetaPageUpdateNextSnapshotId(int grpId, long pageId, long nextSnapshotId) {
+        super(grpId, pageId);
 
         this.nextSnapshotId = nextSnapshotId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index 15d1a0c..f85f468 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@ -45,12 +45,12 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
     private long cntrsPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param allocatedIdxCandidate Page Allocated index candidate
      */
     public MetaPageUpdatePartitionDataRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         long updateCntr,
         long globalRmvId,
@@ -58,7 +58,7 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
         long cntrsPageId, byte state,
         int allocatedIdxCandidate
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         this.updateCntr = updateCntr;
         this.globalRmvId = globalRmvId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
index bdfdef5..f39f91c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
@@ -41,7 +41,7 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
     private long rightChildId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param io IO.
      * @param leftChildId Left child ID.
@@ -49,7 +49,7 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
      * @param rightChildId Right child ID.
      */
     public NewRootInitRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         long newRootId,
         BPlusInnerIO<L> io,
@@ -57,7 +57,7 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
         byte[] rowBytes,
         long rightChildId
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         assert io != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
index ca52bd2..260c2c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
@@ -29,18 +29,18 @@ import org.apache.ignite.internal.util.typedef.internal.U;
  */
 public abstract class PageDeltaRecord extends WALRecord {
     /** */
-    private int cacheId;
+    private int grpId;
 
     /** */
     @GridToStringExclude
     private long pageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      */
-    protected PageDeltaRecord(int cacheId, long pageId) {
-        this.cacheId = cacheId;
+    protected PageDeltaRecord(int grpId, long pageId) {
+        this.grpId = grpId;
         this.pageId = pageId;
     }
 
@@ -52,10 +52,10 @@ public abstract class PageDeltaRecord extends WALRecord {
     }
 
     /**
-     * @return Cache ID.
+     * @return Cache group ID.
      */
-    public int cacheId() {
-        return cacheId;
+    public int groupId() {
+        return grpId;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
index 4756c9a..9c12380 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
@@ -26,11 +26,11 @@ import org.apache.ignite.internal.processors.cache.persistence.freelist.io.Pages
  */
 public class PageListMetaResetCountRecord extends PageDeltaRecord  {
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      */
-    public PageListMetaResetCountRecord(int cacheId, long pageId) {
-        super(cacheId, pageId);
+    public PageListMetaResetCountRecord(int grpId, long pageId) {
+        super(grpId, pageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
index 2577ec3..6c7fc71 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
@@ -33,12 +33,12 @@ public class PagesListAddPageRecord extends PageDeltaRecord {
     private final long dataPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param dataPageId Data page ID to add.
      */
-    public PagesListAddPageRecord(int cacheId, long pageId, long dataPageId) {
-        super(cacheId, pageId);
+    public PagesListAddPageRecord(int grpId, long pageId, long dataPageId) {
+        super(grpId, pageId);
 
         this.dataPageId = dataPageId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
index 673d33c..ee83b8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
@@ -38,13 +38,13 @@ public class PagesListInitNewPageRecord extends InitNewPageRecord {
     private final long addDataPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param prevPageId Previous page ID.
      * @param addDataPageId Optional page ID to add.
      */
     public PagesListInitNewPageRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         int ioType,
         int ioVer,
@@ -52,7 +52,7 @@ public class PagesListInitNewPageRecord extends InitNewPageRecord {
         long prevPageId,
         long addDataPageId
     ) {
-        super(cacheId, pageId, ioType, ioVer, newPageId);
+        super(grpId, pageId, ioType, ioVer, newPageId);
 
         this.prevPageId = prevPageId;
         this.addDataPageId = addDataPageId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
index 2341129..8f2cd72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
@@ -69,6 +69,6 @@ public class PagesListRemovePageRecord extends PageDeltaRecord {
         return S.toString(PagesListRemovePageRecord.class, this,
             "rmvdPageId", U.hexLong(rmvdPageId),
             "pageId", U.hexLong(pageId()),
-            "cacheId", cacheId());
+            "grpId", groupId());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
index 30600e1..f46ac8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
@@ -29,12 +29,12 @@ public class PagesListSetNextRecord extends PageDeltaRecord {
     private final long nextPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param nextPageId Next page ID.
      */
-    public PagesListSetNextRecord(int cacheId, long pageId, long nextPageId) {
-        super(cacheId, pageId);
+    public PagesListSetNextRecord(int grpId, long pageId, long nextPageId) {
+        super(grpId, pageId);
 
         this.nextPageId = nextPageId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
index 590643e..cc27077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
@@ -29,12 +29,12 @@ public class PagesListSetPreviousRecord extends PageDeltaRecord {
     private final long prevPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Page ID.
      * @param prevPageId Previous page ID.
      */
-    public PagesListSetPreviousRecord(int cacheId, long pageId, long prevPageId) {
-        super(cacheId, pageId);
+    public PagesListSetPreviousRecord(int grpId, long pageId, long prevPageId) {
+        super(grpId, pageId);
 
         this.prevPageId = prevPageId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionDestroyRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionDestroyRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionDestroyRecord.java
index c3b8200..834dc58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionDestroyRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionDestroyRecord.java
@@ -24,17 +24,17 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
  */
 public class PartitionDestroyRecord extends WALRecord {
     /** */
-    private int cacheId;
+    private int grpId;
 
     /** */
     private int partId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param partId Partition ID.
      */
-    public PartitionDestroyRecord(int cacheId, int partId) {
-        this.cacheId = cacheId;
+    public PartitionDestroyRecord(int grpId, int partId) {
+        this.grpId = grpId;
         this.partId = partId;
     }
 
@@ -44,17 +44,17 @@ public class PartitionDestroyRecord extends WALRecord {
     }
 
     /**
-     * @return Cache ID.
+     * @return Cache group ID.
      */
-    public int cacheId() {
-        return cacheId;
+    public int groupId() {
+        return grpId;
     }
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      */
-    public void cacheId(int cacheId) {
-        this.cacheId = cacheId;
+    public void groupId(int grpId) {
+        this.grpId = grpId;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionMetaStateRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionMetaStateRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionMetaStateRecord.java
index 95e1a56..8ab794c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionMetaStateRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PartitionMetaStateRecord.java
@@ -27,8 +27,8 @@ public class PartitionMetaStateRecord extends WALRecord {
     /** State. */
     private final byte state;
 
-    /** Cache id. */
-    private final int cacheId;
+    /** Cache group ID. */
+    private final int grpId;
 
     /** Partition id. */
     private final int partId;
@@ -37,11 +37,11 @@ public class PartitionMetaStateRecord extends WALRecord {
     private final long updateCounter;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param state Page ID.
      */
-    public PartitionMetaStateRecord(int cacheId, int partId, GridDhtPartitionState state, long updateCounter) {
-        this.cacheId = cacheId;
+    public PartitionMetaStateRecord(int grpId, int partId, GridDhtPartitionState state, long updateCounter) {
+        this.grpId = grpId;
         this.partId = partId;
         this.state = (byte)state.ordinal();
         this.updateCounter = updateCounter;
@@ -60,10 +60,10 @@ public class PartitionMetaStateRecord extends WALRecord {
     }
 
     /**
-     * @return Cache ID.
+     * @return Cache group ID.
      */
-    public int cacheId() {
-        return cacheId;
+    public int groupId() {
+        return grpId;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
index 92ea88e..b505e4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
@@ -29,12 +29,12 @@ public class RecycleRecord extends PageDeltaRecord {
     private long newPageId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param newPageId New page ID.
      */
-    public RecycleRecord(int cacheId, long pageId, long newPageId) {
-        super(cacheId, pageId);
+    public RecycleRecord(int grpId, long pageId, long newPageId) {
+        super(grpId, pageId);
 
         this.newPageId = newPageId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
index ac5d442..53f9cb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
@@ -34,13 +34,13 @@ public class RemoveRecord extends PageDeltaRecord {
     private int cnt;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param idx Index.
      * @param cnt Count.
      */
-    public RemoveRecord(int cacheId, long pageId, int idx, int cnt) {
-        super(cacheId, pageId);
+    public RemoveRecord(int grpId, long pageId, int idx, int cnt) {
+        super(grpId, pageId);
 
         this.idx = idx;
         this.cnt = cnt;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
index 2f504ad..2fbc0dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
@@ -35,14 +35,14 @@ public class ReplaceRecord<L> extends PageDeltaRecord {
     private int idx;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param io IO.
      * @param rowBytes Row bytes.
      * @param idx Index.
      */
-    public ReplaceRecord(int cacheId, long pageId, BPlusIO<L> io, byte[] rowBytes, int idx) {
-        super(cacheId, pageId);
+    public ReplaceRecord(int grpId, long pageId, BPlusIO<L> io, byte[] rowBytes, int idx) {
+        super(grpId, pageId);
 
         this.io = io;
         this.rowBytes = rowBytes;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
index 5cb3023..87638a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
@@ -37,13 +37,13 @@ public class SplitExistingPageRecord extends PageDeltaRecord {
     private long fwdId;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId  Page ID.
      * @param mid Bisection index.
      * @param fwdId New forward page ID.
      */
-    public SplitExistingPageRecord(int cacheId, long pageId, int mid, long fwdId) {
-        super(cacheId, pageId);
+    public SplitExistingPageRecord(int grpId, long pageId, int mid, long fwdId) {
+        super(grpId, pageId);
 
         this.mid = mid;
         this.fwdId = fwdId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
index 39f2669..7f855f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
@@ -45,7 +45,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
     private int cnt;
 
     /**
-     * @param cacheId Cache ID.
+     * @param grpId Cache group ID.
      * @param pageId Real forward page ID.
      * @param fwdId Virtual forward page ID.
      * @param ioType IO Type.
@@ -55,7 +55,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
      * @param cnt Initial elements count in the page being split.
      */
     public SplitForwardPageRecord(
-        int cacheId,
+        int grpId,
         long pageId,
         long fwdId,
         int ioType,
@@ -64,7 +64,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
         int mid,
         int cnt
     ) {
-        super(cacheId, pageId);
+        super(grpId, pageId);
 
         this.fwdId = fwdId;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index 05e96ff..3d88e8c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@ -35,13 +35,13 @@ public class TrackingPageDeltaRecord extends PageDeltaRecord {
     private final long lastSuccessfulSnapshotId;
 
     /**
-     * @param cacheId Cache id.
+     * @param grpId Cache group id.
      * @param pageId Page id.
      * @param nextSnapshotId
      * @param lastSuccessfulSnapshotId
      */
-    public TrackingPageDeltaRecord(int cacheId, long pageId, long pageIdToMark, long nextSnapshotId, long lastSuccessfulSnapshotId) {
-        super(cacheId, pageId);
+    public TrackingPageDeltaRecord(int grpId, long pageId, long pageIdToMark, long nextSnapshotId, long lastSuccessfulSnapshotId) {
+        super(grpId, pageId);
 
         this.pageIdToMark = pageIdToMark;
         this.nextSnapshotId = nextSnapshotId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
index b2a8f36..9161d69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
@@ -45,7 +45,7 @@ public abstract class DataStructure implements PageLockListener {
     public static Random rnd;
 
     /** */
-    protected final int cacheId;
+    protected final int grpId;
 
     /** */
     protected final PageMemory pageMem;
@@ -57,7 +57,7 @@ public abstract class DataStructure implements PageLockListener {
     protected ReuseList reuseList;
 
     /**
-     * @param cacheId Cache ID.
+     * @param cacheId Cache group ID.
      * @param pageMem Page memory.
      * @param wal Write ahead log manager.
      */
@@ -68,16 +68,16 @@ public abstract class DataStructure implements PageLockListener {
     ) {
         assert pageMem != null;
 
-        this.cacheId = cacheId;
+        this.grpId = cacheId;
         this.pageMem = pageMem;
         this.wal = wal;
     }
 
     /**
-     * @return Cache ID.
+     * @return Cache group ID.
      */
-    public final int getCacheId() {
-        return cacheId;
+    public final int groupId() {
+        return grpId;
     }
 
     /**
@@ -114,7 +114,7 @@ public abstract class DataStructure implements PageLockListener {
      * @throws IgniteCheckedException If failed.
      */
     protected long allocatePageNoReuse() throws IgniteCheckedException {
-        return pageMem.allocatePage(cacheId, PageIdAllocator.INDEX_PARTITION, FLAG_IDX);
+        return pageMem.allocatePage(grpId, PageIdAllocator.INDEX_PARTITION, FLAG_IDX);
     }
 
     /**
@@ -126,7 +126,7 @@ public abstract class DataStructure implements PageLockListener {
         assert PageIdUtils.flag(pageId) == FLAG_IDX && PageIdUtils.partId(pageId) == INDEX_PARTITION ||
             PageIdUtils.flag(pageId) == FLAG_DATA && PageIdUtils.partId(pageId) <= MAX_PARTITION_ID : U.hexLong(pageId);
 
-        return pageMem.acquirePage(cacheId, pageId);
+        return pageMem.acquirePage(grpId, pageId);
     }
 
     /**
@@ -134,7 +134,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param page  Page pointer.
      */
     protected final void releasePage(long pageId, long page) {
-        pageMem.releasePage(cacheId, pageId, page);
+        pageMem.releasePage(grpId, pageId, page);
     }
 
     /**
@@ -143,7 +143,7 @@ public abstract class DataStructure implements PageLockListener {
      * @return Page address or {@code 0} if failed to lock due to recycling.
      */
     protected final long tryWriteLock(long pageId, long page) {
-        return PageHandler.writeLock(pageMem, cacheId, pageId, page, this, true);
+        return PageHandler.writeLock(pageMem, grpId, pageId, page, this, true);
     }
 
     /**
@@ -152,7 +152,7 @@ public abstract class DataStructure implements PageLockListener {
      * @return Page address.
      */
     protected final long writeLock(long pageId, long page) {
-        return PageHandler.writeLock(pageMem, cacheId, pageId, page, this, false);
+        return PageHandler.writeLock(pageMem, grpId, pageId, page, this, false);
     }
 
     /**
@@ -174,7 +174,7 @@ public abstract class DataStructure implements PageLockListener {
      * @return Page address.
      */
     protected final long readLock(long pageId, long page) {
-        return PageHandler.readLock(pageMem, cacheId, pageId, page, this);
+        return PageHandler.readLock(pageMem, grpId, pageId, page, this);
     }
 
     /**
@@ -183,7 +183,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param pageAddr  Page address.
      */
     protected final void readUnlock(long pageId, long page, long pageAddr) {
-        PageHandler.readUnlock(pageMem, cacheId, pageId, page, pageAddr, this);
+        PageHandler.readUnlock(pageMem, grpId, pageId, page, pageAddr, this);
     }
 
     /**
@@ -194,7 +194,7 @@ public abstract class DataStructure implements PageLockListener {
      * @param dirty Dirty flag.
      */
     protected final void writeUnlock(long pageId, long page, long pageAddr, Boolean walPlc, boolean dirty) {
-        PageHandler.writeUnlock(pageMem, cacheId, pageId, page, pageAddr, this, walPlc, dirty);
+        PageHandler.writeUnlock(pageMem, grpId, pageId, page, pageAddr, this, walPlc, dirty);
     }
 
     /**
@@ -204,7 +204,7 @@ public abstract class DataStructure implements PageLockListener {
      * @return {@code true} If we need to make a delta WAL record for the change in this page.
      */
     protected final boolean needWalDeltaRecord(long pageId, long page, Boolean walPlc) {
-        return PageHandler.isWalDeltaRecordNeeded(pageMem, cacheId, pageId, page, wal, walPlc);
+        return PageHandler.isWalDeltaRecordNeeded(pageMem, grpId, pageId, page, wal, walPlc);
     }
 
     /**
@@ -220,7 +220,7 @@ public abstract class DataStructure implements PageLockListener {
         PageHandler<?, R> h,
         int intArg,
         R lockFailed) throws IgniteCheckedException {
-        return PageHandler.writePage(pageMem, cacheId, pageId, this, h, null, null, null, null, intArg, lockFailed);
+        return PageHandler.writePage(pageMem, grpId, pageId, this, h, null, null, null, null, intArg, lockFailed);
     }
 
     /**
@@ -238,7 +238,7 @@ public abstract class DataStructure implements PageLockListener {
         X arg,
         int intArg,
         R lockFailed) throws IgniteCheckedException {
-        return PageHandler.writePage(pageMem, cacheId, pageId, this, h, null, null, null, arg, intArg, lockFailed);
+        return PageHandler.writePage(pageMem, grpId, pageId, this, h, null, null, null, arg, intArg, lockFailed);
     }
 
     /**
@@ -258,7 +258,7 @@ public abstract class DataStructure implements PageLockListener {
         X arg,
         int intArg,
         R lockFailed) throws IgniteCheckedException {
-        return PageHandler.writePage(pageMem, cacheId, pageId, page, this, h, null, null, null, arg, intArg, lockFailed);
+        return PageHandler.writePage(pageMem, grpId, pageId, page, this, h, null, null, null, arg, intArg, lockFailed);
     }
 
     /**
@@ -278,7 +278,7 @@ public abstract class DataStructure implements PageLockListener {
         X arg,
         int intArg,
         R lockFailed) throws IgniteCheckedException {
-        return PageHandler.writePage(pageMem, cacheId, pageId, this, h, init, wal, null, arg, intArg, lockFailed);
+        return PageHandler.writePage(pageMem, grpId, pageId, this, h, init, wal, null, arg, intArg, lockFailed);
     }
 
     /**
@@ -296,7 +296,7 @@ public abstract class DataStructure implements PageLockListener {
         X arg,
         int intArg,
         R lockFailed) throws IgniteCheckedException {
-        return PageHandler.readPage(pageMem, cacheId, pageId, this, h, arg, intArg, lockFailed);
+        return PageHandler.readPage(pageMem, grpId, pageId, this, h, arg, intArg, lockFailed);
     }
 
     /**
@@ -316,7 +316,7 @@ public abstract class DataStructure implements PageLockListener {
         X arg,
         int intArg,
         R lockFailed) throws IgniteCheckedException {
-        return PageHandler.readPage(pageMem, cacheId, pageId, page, this, h, arg, intArg, lockFailed);
+        return PageHandler.readPage(pageMem, grpId, pageId, page, this, h, arg, intArg, lockFailed);
     }
 
     /**
@@ -325,7 +325,7 @@ public abstract class DataStructure implements PageLockListener {
      * @throws IgniteCheckedException if failed.
      */
     protected final void init(long pageId, PageIO init) throws IgniteCheckedException {
-        PageHandler.initPage(pageMem, cacheId, pageId, init, wal, this);
+        PageHandler.initPage(pageMem, grpId, pageId, init, wal, this);
     }
 
     /**
@@ -346,7 +346,7 @@ public abstract class DataStructure implements PageLockListener {
         PageIO.setPageId(pageAddr, rotated);
 
         if (needWalDeltaRecord(pageId, page, walPlc))
-            wal.log(new RecycleRecord(cacheId, pageId, rotated));
+            wal.log(new RecycleRecord(grpId, pageId, rotated));
 
         return rotated;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 8fe9377..5136731 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
@@ -623,7 +623,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     Integer tag
                 ) throws IgniteCheckedException {
                     // First of all, write page to disk.
-                    storeMgr.write(fullId.cacheId(), fullId.pageId(), pageBuf, tag);
+                    storeMgr.write(fullId.groupId(), fullId.pageId(), pageBuf, tag);
 
                     // Only after write we can write page into snapshot.
                     snapshotMgr.flushDirtyPageHandler(fullId, pageBuf, tag);
@@ -1336,25 +1336,25 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                             // Here we do not require tag check because we may be applying memory changes after
                             // several repetitive restarts and the same pages may have changed several times.
-                            int cacheId = pageRec.fullPageId().cacheId();
+                            int grpId = pageRec.fullPageId().groupId();
                             long pageId = pageRec.fullPageId().pageId();
 
-                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(cacheId);
+                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId);
 
-                            long page = pageMem.acquirePage(cacheId, pageId, true);
+                            long page = pageMem.acquirePage(grpId, pageId, true);
 
                             try {
-                                long pageAddr = pageMem.writeLock(cacheId, pageId, page);
+                                long pageAddr = pageMem.writeLock(grpId, pageId, page);
 
                                 try {
                                     PageUtils.putBytes(pageAddr, 0, pageRec.pageData());
                                 }
                                 finally {
-                                    pageMem.writeUnlock(cacheId, pageId, page, null, true, true);
+                                    pageMem.writeUnlock(grpId, pageId, page, null, true, true);
                                 }
                             }
                             finally {
-                                pageMem.releasePage(cacheId, pageId, page);
+                                pageMem.releasePage(grpId, pageId, page);
                             }
 
                             applied++;
@@ -1366,14 +1366,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                         if (apply) {
                             PartitionDestroyRecord destroyRec = (PartitionDestroyRecord)rec;
 
-                            final int cId = destroyRec.cacheId();
+                            final int gId = destroyRec.groupId();
                             final int pId = destroyRec.partitionId();
 
-                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(cId);
+                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(gId);
 
                             pageMem.clearAsync(new P3<Integer, Long, Integer>() {
                                 @Override public boolean apply(Integer cacheId, Long pageId, Integer tag) {
-                                    return cacheId == cId && PageIdUtils.partId(pageId) == pId;
+                                    return cacheId == gId && PageIdUtils.partId(pageId) == pId;
                                 }
                             }, true).get();
                         }
@@ -1384,27 +1384,27 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                         if (apply && rec instanceof PageDeltaRecord) {
                             PageDeltaRecord r = (PageDeltaRecord)rec;
 
-                            int cacheId = r.cacheId();
+                            int grpId = r.groupId();
                             long pageId = r.pageId();
 
-                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(cacheId);
+                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId);
 
                             // Here we do not require tag check because we may be applying memory changes after
                             // several repetitive restarts and the same pages may have changed several times.
-                            long page = pageMem.acquirePage(cacheId, pageId, true);
+                            long page = pageMem.acquirePage(grpId, pageId, true);
 
                             try {
-                                long pageAddr = pageMem.writeLock(cacheId, pageId, page);
+                                long pageAddr = pageMem.writeLock(grpId, pageId, page);
 
                                 try {
                                     r.applyDelta(pageMem, pageAddr);
                                 }
                                 finally {
-                                    pageMem.writeUnlock(cacheId, pageId, page, null, true, true);
+                                    pageMem.writeUnlock(grpId, pageId, page, null, true, true);
                                 }
                             }
                             finally {
-                                pageMem.releasePage(cacheId, pageId, page);
+                                pageMem.releasePage(grpId, pageId, page);
                             }
 
                             applied++;
@@ -1493,7 +1493,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     case PART_META_UPDATE_STATE:
                         PartitionMetaStateRecord metaStateRecord = (PartitionMetaStateRecord)rec;
 
-                        partStates.put(new T2<>(metaStateRecord.cacheId(), metaStateRecord.partitionId()),
+                        partStates.put(new T2<>(metaStateRecord.groupId(), metaStateRecord.partitionId()),
                             new T2<>((int)metaStateRecord.state(), metaStateRecord.updateCounter()));
 
                         break;
@@ -1686,7 +1686,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 if (tag != null) {
                     tmpWriteBuf.rewind();
 
-                    PageStore store = storeMgr.writeInternal(fullId.cacheId(), fullId.pageId(), tmpWriteBuf, tag);
+                    PageStore store = storeMgr.writeInternal(fullId.groupId(), fullId.pageId(), tmpWriteBuf, tag);
 
                     tmpWriteBuf.rewind();
 
@@ -2326,7 +2326,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                     snapshotMgr.beforePageWrite(fullId);
 
-                    int grpId = fullId.cacheId();
+                    int grpId = fullId.groupId();
 
                     CacheGroupContext grp = context().cache().cacheGroup(grpId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
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 4d30eb4..bd902fb 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
@@ -888,7 +888,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                         reuseRoot.pageId().pageId(),
                         reuseRoot.isAllocated()) {
                         @Override protected long allocatePageNoReuse() throws IgniteCheckedException {
-                            return pageMem.allocatePage(cacheId, partId, PageIdAllocator.FLAG_DATA);
+                            return pageMem.allocatePage(grpId, partId, PageIdAllocator.FLAG_DATA);
                         }
                     };
 
@@ -904,7 +904,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                         treeRoot.pageId().pageId(),
                         treeRoot.isAllocated()) {
                         @Override protected long allocatePageNoReuse() throws IgniteCheckedException {
-                            return pageMem.allocatePage(cacheId, partId, PageIdAllocator.FLAG_DATA);
+                            return pageMem.allocatePage(grpId, partId, PageIdAllocator.FLAG_DATA);
                         }
                     };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
index 806afb8..743f3b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
@@ -186,7 +186,7 @@ public class MetadataStorage implements MetaStore {
 
         /** {@inheritDoc} */
         @Override protected long allocatePageNoReuse() throws IgniteCheckedException {
-            return pageMem.allocatePage(getCacheId(), allocPartId, allocSpace);
+            return pageMem.allocatePage(groupId(), allocPartId, allocSpace);
         }
 
         /** {@inheritDoc} */


[07/41] ignite git commit: Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2

Posted by sb...@apache.org.
Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2


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

Branch: refs/heads/ignite-5578-1
Commit: 984ab5bd3c08b2af9222964451acc281b92814b8
Parents: 4fc8124 99fd75d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 14:43:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 14:43:50 2017 +0300

----------------------------------------------------------------------
 .../pagemem/snapshot/SnapshotOperation.java     | 17 +++++-
 .../cache/CacheAffinitySharedManager.java       | 10 ++--
 .../processors/cache/ClusterCachesInfo.java     |  2 +-
 .../processors/cache/ExchangeActions.java       | 34 ++++++------
 .../processors/cache/GridCacheProcessor.java    |  2 +-
 .../GridDhtPartitionsExchangeFuture.java        |  2 +-
 .../IgniteChangeGlobalStateFailOverTest.java    | 56 +++++++++++++-------
 .../query/h2/DmlStatementsProcessor.java        | 19 +++++++
 .../processors/query/h2/H2DatabaseType.java     | 10 ++++
 .../processors/query/h2/H2RowDescriptor.java    | 13 +++++
 10 files changed, 120 insertions(+), 45 deletions(-)
----------------------------------------------------------------------



[13/41] ignite git commit: Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2

Posted by sb...@apache.org.
Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2


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

Branch: refs/heads/ignite-5578-1
Commit: ea4420e6aad509c4ae58d6d0880918c98fd1959a
Parents: 13399d5 a9387ad
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 15:24:26 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 15:24:26 2017 +0300

----------------------------------------------------------------------
 .../Cache/PersistentStoreTest.cs                | 36 ++++++++++++++++++--
 1 file changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[36/41] ignite git commit: GG-12418 - WAL hangs on any error during segment rollover

Posted by sb...@apache.org.
GG-12418 - WAL hangs on any error during segment rollover


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

Branch: refs/heads/ignite-5578-1
Commit: 17d881ba0122a7f90cac9846c376300a1d001bdd
Parents: f1c8e59
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Mon Jul 10 13:55:47 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 10 13:57:51 2017 +0300

----------------------------------------------------------------------
 .../PersistentStoreConfiguration.java           |  24 +++
 .../cache/persistence/file/FileIO.java          | 154 +++++++++++++++
 .../cache/persistence/file/FileIODecorator.java |  98 ++++++++++
 .../cache/persistence/file/FileIOFactory.java   |  45 +++++
 .../cache/persistence/file/FilePageStore.java   |  51 +++--
 .../persistence/file/FilePageStoreManager.java  |   2 +
 .../persistence/file/RandomAccessFileIO.java    | 110 +++++++++++
 .../file/RandomAccessFileIOFactory.java         |  42 ++++
 .../wal/AbstractWalRecordsIterator.java         |  22 ++-
 .../cache/persistence/wal/FileInput.java        |  40 ++--
 .../wal/FileWriteAheadLogManager.java           | 161 ++++++++-------
 .../wal/reader/IgniteWalIteratorFactory.java    |  13 +-
 .../wal/reader/StandaloneGridKernalContext.java |  15 +-
 .../reader/StandaloneIgnitePluginProcessor.java |  38 ++++
 .../reader/StandaloneWalRecordsIterator.java    |  37 ++--
 ...gnitePdsRecoveryAfterFileCorruptionTest.java |  11 +-
 .../db/wal/IgniteWalFlushFailoverTest.java      | 195 +++++++++++++++++++
 .../db/wal/crc/IgniteDataIntegrityTests.java    |  10 +-
 .../db/wal/reader/IgniteWalReaderTest.java      |   9 +-
 .../db/wal/reader/MockWalIteratorFactory.java   |   8 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   4 +
 21 files changed, 919 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index b531f9d..4792483 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -16,6 +16,8 @@
  */
 package org.apache.ignite.configuration;
 
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 import java.io.Serializable;
@@ -133,6 +135,9 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Always write full pages. */
     private boolean alwaysWriteFullPages = DFLT_WAL_ALWAYS_WRITE_FULL_PAGES;
 
+    /** Factory to provide I/O interface for files */
+    private FileIOFactory fileIOFactory = new RandomAccessFileIOFactory();
+
     /**
      * Number of sub-intervals the whole {@link #setRateTimeInterval(long)} will be split into to calculate
      * rate-based metrics.
@@ -539,6 +544,25 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
+     * Factory to provide implementation of FileIO interface
+     * which is used for any file read/write operations
+     *
+     * @return File I/O factory
+     */
+    public FileIOFactory getFileIOFactory() {
+        return fileIOFactory;
+    }
+
+    /**
+     * @param fileIOFactory File I/O factory
+     */
+    public PersistentStoreConfiguration setFileIOFactory(FileIOFactory fileIOFactory) {
+        this.fileIOFactory = fileIOFactory;
+
+        return this;
+    }
+
+    /**
      * <b>Note:</b> setting this value with {@link WALMode#DEFAULT} may generate file size overhead for WAL segments in case
      * grid is used rarely.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
new file mode 100644
index 0000000..1e81150
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIO.java
@@ -0,0 +1,154 @@
+/*
+ * 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.persistence.file;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Interface to perform file I/O operations.
+ */
+public interface FileIO extends AutoCloseable {
+    /**
+     * Returns current file position.
+     *
+     * @return  Current file position,
+     *          a non-negative integer counting the number of bytes
+     *          from the beginning of the file to the current position.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public long position() throws IOException;
+
+    /**
+     * Sets new current file position.
+     *
+     * @param  newPosition
+     *         The new position, a non-negative integer counting
+     *         the number of bytes from the beginning of the file.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public void position(long newPosition) throws IOException;
+
+    /**
+     * Reads a sequence of bytes from this file into the {@code destinationBuffer}.
+     *
+     * @param destinationBuffer Destination byte buffer.
+     *
+     * @return Number of read bytes.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public int read(ByteBuffer destinationBuffer) throws IOException;
+
+    /**
+     * Reads a sequence of bytes from this file into the {@code destinationBuffer}
+     * starting from specified file {@code position}.
+     *
+     * @param destinationBuffer Destination byte buffer.
+     * @param position Starting position of file.
+     *
+     * @return Number of read bytes.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public int read(ByteBuffer destinationBuffer, long position) throws IOException;
+
+    /**
+     * Reads a up to {@code length} bytes from this file into the {@code buffer}.
+     *
+     * @param buffer Destination byte array.
+     * @param offset The start offset in array {@code b}
+     *               at which the data is written.
+     * @param length Maximum number of bytes read.
+     *
+     * @return Number of read bytes.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public int read(byte[] buffer, int offset, int length) throws IOException;
+
+    /**
+     * Writes a sequence of bytes to this file from the {@code sourceBuffer}.
+     *
+     * @param sourceBuffer Source buffer.
+     *
+     * @return Number of written bytes.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public int write(ByteBuffer sourceBuffer) throws IOException;
+
+    /**
+     * Writes a sequence of bytes to this file from the {@code sourceBuffer}
+     * starting from specified file {@code position}
+     *
+     * @param sourceBuffer Source buffer.
+     * @param position Starting file position.
+     *
+     * @return Number of written bytes.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public int write(ByteBuffer sourceBuffer, long position) throws IOException;
+
+    /**
+     * Writes {@code length} bytes from the {@code buffer}
+     * starting at offset {@code off} to this file.
+     *
+     * @param buffer Source byte array.
+     * @param offset Start offset in the {@code buffer}.
+     * @param length Number of bytes to write.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public void write(byte[] buffer, int offset, int length) throws IOException;
+
+    /**
+     * Forces any updates of this file to be written to the storage
+     * device that contains it.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public void force() throws IOException;
+
+    /**
+     * Returns current file size in bytes.
+     *
+     * @return File size.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public long size() throws IOException;
+
+    /**
+     * Truncates current file to zero length
+     * and resets current file position to zero.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    public void clear() throws IOException;
+
+    /**
+     * Closes current file.
+     *
+     * @throws IOException If some I/O error occurs.
+     */
+    @Override public void close() throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
new file mode 100644
index 0000000..3e80ef8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIODecorator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.persistence.file;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Decorator class for File I/O
+ */
+public class FileIODecorator implements FileIO {
+
+    /** File I/O delegate */
+    private final FileIO delegate;
+
+    /**
+     *
+     * @param delegate File I/O delegate
+     */
+    public FileIODecorator(FileIO delegate) {
+        this.delegate = delegate;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long position() throws IOException {
+        return delegate.position();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void position(long newPosition) throws IOException {
+        delegate.position(newPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(ByteBuffer destinationBuffer) throws IOException {
+        return delegate.read(destinationBuffer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(ByteBuffer destinationBuffer, long position) throws IOException {
+        return delegate.read(destinationBuffer, position);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(byte[] buffer, int offset, int length) throws IOException {
+        return delegate.read(buffer, offset, length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int write(ByteBuffer sourceBuffer) throws IOException {
+        return delegate.write(sourceBuffer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int write(ByteBuffer sourceBuffer, long position) throws IOException {
+        return delegate.write(sourceBuffer, position);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(byte[] buffer, int offset, int length) throws IOException {
+        delegate.write(buffer, offset, length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void force() throws IOException {
+        delegate.force();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size() throws IOException {
+        return delegate.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() throws IOException {
+        delegate.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        delegate.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
new file mode 100644
index 0000000..0ffc653
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileIOFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.persistence.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+
+public interface FileIOFactory extends Serializable {
+
+    /**
+     * Creates I/O interface for file with default I/O mode
+     *
+     * @param file File
+     * @return File I/O interface
+     * @throws IOException If I/O interface creation was failed
+     */
+    FileIO create(File file) throws IOException;
+
+    /**
+     * Creates I/O interface for file with specified mode
+     *
+     * @param file File
+     * @param mode I/O mode in
+     * @return File I/O interface
+     * @throws IOException If I/O interface creation was failed
+     */
+    FileIO create(File file, String mode) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
index 6ddc9fc..c827e96 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
@@ -19,10 +19,8 @@ package org.apache.ignite.internal.processors.cache.persistence.file;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -61,11 +59,11 @@ public class FilePageStore implements PageStore {
     /** Database configuration. */
     private final MemoryConfiguration dbCfg;
 
-    /** */
-    private RandomAccessFile file;
+    /** Factory to provide I/O interfaces for read/write operations with files */
+    private final FileIOFactory ioFactory;
 
-    /** */
-    private FileChannel ch;
+    /** I/O interface for read/write operations with file */
+    private FileIO fileIO;
 
     /** */
     private final AtomicLong allocated;
@@ -91,11 +89,12 @@ public class FilePageStore implements PageStore {
     /**
      * @param file File.
      */
-    public FilePageStore(byte type, File file, MemoryConfiguration cfg) {
+    public FilePageStore(byte type, File file, FileIOFactory factory, MemoryConfiguration cfg) {
         this.type = type;
 
         cfgFile = file;
         dbCfg = cfg;
+        ioFactory = factory;
 
         allocated = new AtomicLong();
 
@@ -136,7 +135,7 @@ public class FilePageStore implements PageStore {
             ByteBuffer hdr = header(type, dbCfg.getPageSize());
 
             while (hdr.remaining() > 0)
-                ch.write(hdr);
+                fileIO.write(hdr);
         }
         catch (IOException e) {
             throw new IgniteException("Check file failed.", e);
@@ -154,7 +153,7 @@ public class FilePageStore implements PageStore {
             ByteBuffer hdr = ByteBuffer.allocate(HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN);
 
             while (hdr.remaining() > 0)
-                ch.read(hdr);
+                fileIO.read(hdr);
 
             hdr.rewind();
 
@@ -186,7 +185,7 @@ public class FilePageStore implements PageStore {
                     " [expectedPageSize=" + dbCfg.getPageSize() +
                     ", filePageSize=" + pageSize + "]");
 
-            long fileSize = file.length();
+            long fileSize = cfgFile.length();
 
             if (fileSize == HEADER_SIZE) // Every file has a special meta page.
                 fileSize = pageSize + HEADER_SIZE;
@@ -214,9 +213,9 @@ public class FilePageStore implements PageStore {
             if (!inited)
                 return;
 
-            ch.force(false);
+            fileIO.force();
 
-            file.close();
+            fileIO.close();
 
             if (cleanFile)
                 cfgFile.delete();
@@ -241,9 +240,7 @@ public class FilePageStore implements PageStore {
 
             this.tag = tag;
 
-            ch.position(0);
-
-            file.setLength(0);
+            fileIO.clear();
 
             allocated.set(initFile());
         }
@@ -277,7 +274,7 @@ public class FilePageStore implements PageStore {
 
         try {
             if (inited)
-                allocated.set(ch.size());
+                allocated.set(fileIO.size());
 
             recover = false;
         }
@@ -303,7 +300,7 @@ public class FilePageStore implements PageStore {
             int len = pageSize;
 
             do {
-                int n = ch.read(pageBuf, off);
+                int n = fileIO.read(pageBuf, off);
 
                 // If page was not written yet, nothing to read.
                 if (n < 0) {
@@ -330,7 +327,7 @@ public class FilePageStore implements PageStore {
                 if ((savedCrc32 ^ curCrc32) != 0)
                     throw new IgniteDataIntegrityViolationException("Failed to read page (CRC validation failed) " +
                         "[id=" + U.hexLong(pageId) + ", off=" + (off - pageSize) +
-                        ", file=" + cfgFile.getAbsolutePath() + ", fileSize=" + ch.size() +
+                        ", file=" + cfgFile.getAbsolutePath() + ", fileSize=" + fileIO.size() +
                         ", savedCrc=" + U.hexInt(savedCrc32) + ", curCrc=" + U.hexInt(curCrc32) + "]");
             }
 
@@ -356,7 +353,7 @@ public class FilePageStore implements PageStore {
             long off = 0;
 
             do {
-                int n = ch.read(buf, off);
+                int n = fileIO.read(buf, off);
 
                 // If page was not written yet, nothing to read.
                 if (n < 0)
@@ -382,16 +379,14 @@ public class FilePageStore implements PageStore {
 
             try {
                 if (!inited) {
-                    RandomAccessFile rndFile = null;
+                    FileIO fileIO = null;
 
                     IgniteCheckedException err = null;
 
                     try {
-                        file = rndFile = new RandomAccessFile(cfgFile, "rw");
-
-                        ch = file.getChannel();
+                        this.fileIO = fileIO = ioFactory.create(cfgFile, "rw");
 
-                        if (file.length() == 0)
+                        if (cfgFile.length() == 0)
                             allocated.set(initFile());
                         else
                             allocated.set(checkFile());
@@ -402,9 +397,9 @@ public class FilePageStore implements PageStore {
                         throw err = new IgniteCheckedException("Can't open file: " + cfgFile.getName(), e);
                     }
                     finally {
-                        if (err != null && rndFile != null)
+                        if (err != null && fileIO != null)
                             try {
-                                rndFile.close();
+                                fileIO.close();
                             }
                             catch (IOException e) {
                                 err.addSuppressed(e);
@@ -447,7 +442,7 @@ public class FilePageStore implements PageStore {
             int len = pageSize;
 
             do {
-                int n = ch.write(pageBuf, off);
+                int n = fileIO.write(pageBuf, off);
 
                 off += n;
 
@@ -478,7 +473,7 @@ public class FilePageStore implements PageStore {
         try {
             init();
 
-            ch.force(false);
+            fileIO.force();
         }
         catch (IOException e) {
             throw new IgniteCheckedException("Sync error", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index 6aa2243..4a56ec7 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -367,6 +367,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
         FilePageStore idxStore = new FilePageStore(
             PageMemory.FLAG_IDX,
             idxFile,
+            pstCfg.getFileIOFactory(),
             cctx.kernalContext().config().getMemoryConfiguration());
 
         FilePageStore[] partStores = new FilePageStore[grpDesc.config().getAffinity().partitions()];
@@ -375,6 +376,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
             FilePageStore partStore = new FilePageStore(
                 PageMemory.FLAG_DATA,
                 new File(cacheWorkDir, String.format(PART_FILE_TEMPLATE, partId)),
+                pstCfg.getFileIOFactory(),
                 cctx.kernalContext().config().getMemoryConfiguration()
             );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
new file mode 100644
index 0000000..73a560a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIO.java
@@ -0,0 +1,110 @@
+/*
+ * 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.persistence.file;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+/**
+ * File I/O implementation based on {@code java.io.RandomAccessFile}.
+ */
+public class RandomAccessFileIO implements FileIO {
+
+    /**
+     * Random access file associated with this I/O
+     */
+    private final RandomAccessFile file;
+
+    /**
+     * File channel associated with {@code file}
+     */
+    private final FileChannel channel;
+
+    /**
+     * Creates I/O implementation for specified {@code file}
+     *
+     * @param file Random access file
+     */
+    public RandomAccessFileIO(RandomAccessFile file) {
+        this.file = file;
+        this.channel = file.getChannel();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long position() throws IOException {
+        return channel.position();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void position(long newPosition) throws IOException {
+        channel.position(newPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(ByteBuffer destinationBuffer) throws IOException {
+        return channel.read(destinationBuffer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(ByteBuffer destinationBuffer, long position) throws IOException {
+        return channel.read(destinationBuffer, position);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(byte[] buffer, int offset, int length) throws IOException {
+        return file.read(buffer, offset, length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int write(ByteBuffer sourceBuffer) throws IOException {
+        return channel.write(sourceBuffer);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int write(ByteBuffer sourceBuffer, long position) throws IOException {
+        return channel.write(sourceBuffer, position);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(byte[] buffer, int offset, int length) throws IOException {
+        file.write(buffer, offset, length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void force() throws IOException {
+        channel.force(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long size() throws IOException {
+        return channel.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void clear() throws IOException {
+        channel.position(0);
+        file.setLength(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        file.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
new file mode 100644
index 0000000..6b731f2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/RandomAccessFileIOFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.persistence.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+/**
+ * File I/O factory which provides RandomAccessFileIO implementation of FileIO.
+ */
+public class RandomAccessFileIOFactory implements FileIOFactory {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public FileIO create(File file) throws IOException {
+        return create(file, "rw");
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileIO create(File file, String mode) throws IOException {
+        RandomAccessFile rf = new RandomAccessFile(file, mode);
+
+        return new RandomAccessFileIO(rf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
index f4bace1..beed90b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/AbstractWalRecordsIterator.java
@@ -21,15 +21,15 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -71,6 +71,9 @@ public abstract class AbstractWalRecordsIterator
     /** Serializer of current version to read headers. */
     @NotNull private final RecordSerializer serializer;
 
+    /** Factory to provide I/O interfaces for read/write operations with files */
+    @NotNull protected final FileIOFactory ioFactory;
+
     /** Utility buffer for reading records */
     private final ByteBufferExpander buf;
 
@@ -84,11 +87,13 @@ public abstract class AbstractWalRecordsIterator
         @NotNull final IgniteLogger log,
         @NotNull final GridCacheSharedContext sharedCtx,
         @NotNull final RecordSerializer serializer,
+        @NotNull final FileIOFactory ioFactory,
         final int bufSize
     ) {
         this.log = log;
         this.sharedCtx = sharedCtx;
         this.serializer = serializer;
+        this.ioFactory = ioFactory;
 
         // Do not allocate direct buffer for iterator.
         buf = new ByteBufferExpander(bufSize, ByteOrder.nativeOrder());
@@ -229,15 +234,14 @@ public abstract class AbstractWalRecordsIterator
         @Nullable final FileWALPointer start)
         throws IgniteCheckedException, FileNotFoundException {
         try {
-            RandomAccessFile rf = new RandomAccessFile(desc.file, "r");
+            FileIO fileIO = ioFactory.create(desc.file, "r");
 
             try {
-                FileChannel ch = rf.getChannel();
-                FileInput in = new FileInput(ch, buf);
+                FileInput in = new FileInput(fileIO, buf);
 
                 // Header record must be agnostic to the serializer version.
                 WALRecord rec = serializer.readRecord(in,
-                    new FileWALPointer(desc.idx, (int)ch.position(), 0));
+                    new FileWALPointer(desc.idx, (int)fileIO.position(), 0));
 
                 if (rec == null)
                     return null;
@@ -252,11 +256,11 @@ public abstract class AbstractWalRecordsIterator
                 if (start != null && desc.idx == start.index())
                     in.seek(start.fileOffset());
 
-                return new FileWriteAheadLogManager.ReadFileHandle(rf, desc.idx, sharedCtx.igniteInstanceName(), ser, in);
+                return new FileWriteAheadLogManager.ReadFileHandle(fileIO, desc.idx, sharedCtx.igniteInstanceName(), ser, in);
             }
             catch (SegmentEofException | EOFException ignore) {
                 try {
-                    rf.close();
+                    fileIO.close();
                 }
                 catch (IOException ce) {
                     throw new IgniteCheckedException(ce);
@@ -266,7 +270,7 @@ public abstract class AbstractWalRecordsIterator
             }
             catch (IOException | IgniteCheckedException e) {
                 try {
-                    rf.close();
+                    fileIO.close();
                 }
                 catch (IOException ce) {
                     e.addSuppressed(ce);

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
index 00c7c02..6443a7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileInput.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.persistence.wal;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.jetbrains.annotations.NotNull;
@@ -36,8 +36,8 @@ public final class FileInput implements ByteBufferBackedDataInput {
      */
     private ByteBuffer buf;
 
-    /** File channel to read chunks from */
-    private FileChannel ch;
+    /** I/O interface for read/write operations with file */
+    private FileIO io;
 
     /** */
     private long pos;
@@ -46,28 +46,20 @@ public final class FileInput implements ByteBufferBackedDataInput {
     private ByteBufferExpander expBuf;
 
     /**
-     * @param ch  Channel to read from
-     * @param buf Buffer for reading blocks of data into
+     * @param io FileIO to read from.
+     * @param buf Buffer for reading blocks of data into.
      */
-    public FileInput(FileChannel ch, ByteBuffer buf) throws IOException {
-        assert ch != null;
+    public FileInput(FileIO io, ByteBufferExpander buf) throws IOException {
+        assert io != null;
 
-        this.ch = ch;
-        this.buf = buf;
+        this.io = io;
+        this.buf = buf.buffer();
 
-        pos = ch.position();
+        expBuf = buf;
 
-        clearBuffer();
-    }
-
-    /**
-     * @param ch Channel to read from
-     * @param expBuf ByteBufferWrapper with ability expand buffer dynamically.
-     */
-    public FileInput(FileChannel ch, ByteBufferExpander expBuf) throws IOException {
-        this(ch, expBuf.buffer());
+        pos = io.position();
 
-        this.expBuf = expBuf;
+        clearBuffer();
     }
 
     /**
@@ -84,10 +76,10 @@ public final class FileInput implements ByteBufferBackedDataInput {
      * @param pos Position in bytes from file begin.
      */
     public void seek(long pos) throws IOException {
-        if (pos > ch.size())
+        if (pos > io.size())
             throw new EOFException();
 
-        ch.position(pos);
+        io.position(pos);
 
         this.pos = pos;
 
@@ -118,10 +110,10 @@ public final class FileInput implements ByteBufferBackedDataInput {
         buf.compact();
 
         do {
-            int read = ch.read(buf);
+            int read = io.read(buf);
 
             if (read == -1)
-                throw new EOFException("EOF at position [" + ch.position() + "] expected to read [" + requested + "] bytes");
+                throw new EOFException("EOF at position [" + io.position() + "] expected to read [" + requested + "] bytes");
 
             available += read;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 162f43d..5c112fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -22,10 +22,8 @@ import java.io.File;
 import java.io.FileFilter;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
 import java.nio.file.Files;
 import java.sql.Time;
 import java.util.Arrays;
@@ -48,6 +46,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.events.WalSegmentArchivedEvent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
@@ -61,7 +60,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.PersistenceMetricsImpl;
-import org.apache.ignite.events.WalSegmentArchivedEvent;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
@@ -153,6 +153,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** */
     private volatile long oldestArchiveSegmentIdx;
 
+    /** Factory to provide I/O interfaces for read/write operations with files */
+    private final FileIOFactory ioFactory;
+
     /** Updater for {@link #currentHnd}, used for verify there are no concurrent update for current log segment handle */
     private static final AtomicReferenceFieldUpdater<FileWriteAheadLogManager, FileWriteHandle> currentHndUpd =
         AtomicReferenceFieldUpdater.newUpdater(FileWriteAheadLogManager.class, FileWriteHandle.class, "currentHnd");
@@ -181,6 +184,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** Current log segment handle */
     private volatile FileWriteHandle currentHnd;
 
+    /** Environment failure. */
+    private volatile Throwable envFailed;
+
     /**
      * Positive (non-0) value indicates WAL can be archived even if not complete<br>
      * See {@link PersistentStoreConfiguration#setWalAutoArchiveAfterInactivity(long)}<br>
@@ -225,6 +231,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         flushFreq = psCfg.getWalFlushFrequency();
         fsyncDelay = psCfg.getWalFsyncDelay();
         alwaysWriteFullPages = psCfg.isAlwaysWriteFullPages();
+        ioFactory = psCfg.getFileIOFactory();
         walAutoArchiveAfterInactivity = psCfg.getWalAutoArchiveAfterInactivity();
         evt = ctx.event();
     }
@@ -322,7 +329,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 archiver.shutdown();
         }
         catch (Exception e) {
-            U.error(log, "Failed to gracefully close WAL segment: " + currHnd.file, e);
+            U.error(log, "Failed to gracefully close WAL segment: " + currentHnd.fileIO, e);
         }
     }
 
@@ -493,6 +500,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 return ptr;
             }
 
+            checkEnvironment();
+
             if (isStopping())
                 throw new IgniteCheckedException("Stopping.");
         }
@@ -549,6 +558,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             end,
             psCfg,
             serializer,
+            ioFactory,
             archiver,
             log,
             tlbSize
@@ -800,13 +810,13 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         int len = lastReadPtr == null ? 0 : lastReadPtr.length();
 
         try {
-            RandomAccessFile file = new RandomAccessFile(curFile, "rw");
+            FileIO fileIO = ioFactory.create(curFile);
 
             try {
                 // readSerializerVersion will change the channel position.
                 // This is fine because the FileWriteHandle consitructor will move it
                 // to offset + len anyways.
-                int serVer = readSerializerVersion(file, curFile, absIdx);
+                int serVer = readSerializerVersion(fileIO, curFile, absIdx);
 
                 RecordSerializer ser = forVersion(cctx, serVer);
 
@@ -815,7 +825,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                         ", offset=" + offset + ", ver=" + serVer + ']');
 
                 FileWriteHandle hnd = new FileWriteHandle(
-                    file,
+                    fileIO,
                     absIdx,
                     cctx.igniteInstanceName(),
                     offset + len,
@@ -835,7 +845,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 return hnd;
             }
             catch (IgniteCheckedException | IOException e) {
-                file.close();
+                fileIO.close();
 
                 throw e;
             }
@@ -862,10 +872,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             if (log.isDebugEnabled())
                 log.debug("Switching to a new WAL segment: " + nextFile.getAbsolutePath());
 
-            RandomAccessFile file = new RandomAccessFile(nextFile, "rw");
+            FileIO fileIO = ioFactory.create(nextFile);
 
             FileWriteHandle hnd = new FileWriteHandle(
-                file,
+                fileIO,
                 curIdx + 1,
                 cctx.igniteInstanceName(),
                 0,
@@ -929,22 +939,22 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         if (log.isDebugEnabled())
             log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']');
 
-        try (RandomAccessFile rnd = new RandomAccessFile(file, "rw")) {
+        try (FileIO fileIO = ioFactory.create(file, "rw")) {
             int left = psCfg.getWalSegmentSize();
 
             if (mode == WALMode.DEFAULT) {
                 while (left > 0) {
                     int toWrite = Math.min(FILL_BUF.length, left);
 
-                    rnd.write(FILL_BUF, 0, toWrite);
+                    fileIO.write(FILL_BUF, 0, toWrite);
 
                     left -= toWrite;
                 }
 
-                rnd.getChannel().force(false);
+                fileIO.force();
             }
             else
-                rnd.setLength(0);
+                fileIO.clear();
         }
         catch (IOException e) {
             throw new IgniteCheckedException("Failed to format WAL segment file: " + file.getAbsolutePath(), e);
@@ -1033,6 +1043,15 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
+     * @throws StorageException If environment is no longer valid and we missed a WAL write.
+     */
+    private void checkEnvironment() throws StorageException {
+        if (envFailed != null)
+            throw new StorageException("Failed to flush WAL buffer (environment was invalidated by a " +
+                    "previous error)", envFailed);
+    }
+
+    /**
      * File archiver operates on absolute segment indexes. For any given absolute segment index N we can calculate
      * the work WAL segment: S(N) = N % psCfg.walSegments.
      * When a work segment is finished, it is given to the archiver. If the absolute index of last archived segment
@@ -1337,8 +1356,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 Files.move(dstTmpFile.toPath(), dstFile.toPath());
 
                 if (mode == WALMode.DEFAULT) {
-                    try (RandomAccessFile f0 = new RandomAccessFile(dstFile, "rw")) {
-                        f0.getChannel().force(false);
+                    try (FileIO f0 = ioFactory.create(dstFile, "rw")) {
+                        f0.force();
                     }
                 }
             }
@@ -1402,20 +1421,21 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
-     * @param rf Random access file.
+     * @param io I/O interface for file.
      * @param file File object.
      * @param idx File index to read.
      * @return Serializer version stored in the file.
      * @throws IOException If failed to read serializer version.
      * @throws IgniteCheckedException If failed to read serializer version.
      */
-    private int readSerializerVersion(RandomAccessFile rf, File file, long idx)
+    private int readSerializerVersion(FileIO io, File file, long idx)
         throws IOException, IgniteCheckedException {
         try {
             ByteBuffer buf = ByteBuffer.allocate(RecordV1Serializer.HEADER_RECORD_SIZE);
             buf.order(ByteOrder.nativeOrder());
 
-            FileInput in = new FileInput(rf.getChannel(), buf);
+            FileInput in = new FileInput(io,
+                new ByteBufferExpander(RecordV1Serializer.HEADER_RECORD_SIZE, ByteOrder.nativeOrder()));
 
             // Header record must be agnostic to the serializer version.
             WALRecord rec = serializer.readRecord(in, new FileWALPointer(idx, 0, 0));
@@ -1541,11 +1561,8 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      *
      */
     private abstract static class FileHandle {
-        /** */
-        protected RandomAccessFile file;
-
-        /** */
-        protected FileChannel ch;
+        /** I/O interface for read/write operations with file */
+        protected FileIO fileIO;
 
         /** Absolute WAL segment file index (incremental counter) */
         protected final long idx;
@@ -1554,17 +1571,13 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         protected String gridName;
 
         /**
-         * @param file File.
+         * @param fileIO I/O interface for read/write operations of FileHandle.
          * @param idx Absolute WAL segment file index (incremental counter).
          */
-        private FileHandle(RandomAccessFile file, long idx, String gridName) {
-            this.file = file;
+        private FileHandle(FileIO fileIO, long idx, String gridName) {
+            this.fileIO = fileIO;
             this.idx = idx;
             this.gridName = gridName;
-
-            ch = file.getChannel();
-
-            assert ch != null;
         }
     }
 
@@ -1585,19 +1598,19 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         private boolean workDir;
 
         /**
-         * @param file File to read.
+         * @param fileIO I/O interface for read/write operations of FileHandle.
          * @param idx Absolute WAL segment file index (incremental counter).
          * @param ser Entry serializer.
          * @param in File input.
          */
         ReadFileHandle(
-            RandomAccessFile file,
-            long idx,
-            String gridName,
-            RecordSerializer ser,
-            FileInput in
+                FileIO fileIO,
+                long idx,
+                String gridName,
+                RecordSerializer ser,
+                FileInput in
         ) {
-            super(file, idx, gridName);
+            super(fileIO, idx, gridName);
 
             this.ser = ser;
             this.in = in;
@@ -1608,7 +1621,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          */
         public void close() throws IgniteCheckedException {
             try {
-                file.close();
+                fileIO.close();
             }
             catch (IOException e) {
                 throw new IgniteCheckedException(e);
@@ -1644,10 +1657,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** */
         private volatile long lastFsyncPos;
 
-        /** Environment failure. */
-        private volatile Throwable envFailed;
-
-        /** Stop guard to provide warranty that only one thread will be successful in calling {@link #close(boolean)} */
+        /** Stop guard to provide warranty that only one thread will be successful in calling {@link #close(boolean)}*/
         private final AtomicBoolean stop = new AtomicBoolean(false);
 
         /** */
@@ -1661,12 +1671,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
         /**
          * Next segment available condition.
-         * Protection from "spurious wakeup" is provided by predicate {@link #ch}=<code>null</code>
+         * Protection from "spurious wakeup" is provided by predicate {@link #fileIO}=<code>null</code>
          */
         private final Condition nextSegment = lock.newCondition();
 
         /**
-         * @param file Mapped file to use.
+         * @param fileIO I/O file interface to use
          * @param idx Absolute WAL segment file index for easy access.
          * @param pos Position.
          * @param maxSegmentSize Max segment size.
@@ -1674,18 +1684,18 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * @throws IOException If failed.
          */
         private FileWriteHandle(
-            RandomAccessFile file,
+            FileIO fileIO,
             long idx,
             String gridName,
             long pos,
             long maxSegmentSize,
             RecordSerializer serializer
         ) throws IOException {
-            super(file, idx, gridName);
+            super(fileIO, idx, gridName);
 
             assert serializer != null;
 
-            ch.position(pos);
+            fileIO.position(pos);
 
             this.maxSegmentSize = maxSegmentSize;
             this.serializer = serializer;
@@ -1887,6 +1897,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             catch (Throwable e) {
                 invalidateEnvironment(e);
 
+                // All workers waiting for a next segment must be woken up and stopped
+                signalNextAvailable();
+
                 throw e;
             }
         }
@@ -1990,7 +2003,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                     long start = metricsEnabled ? System.nanoTime() : 0;
 
                     try {
-                        ch.force(false);
+                        fileIO.force();
                     }
                     catch (IOException e) {
                         throw new StorageException(e);
@@ -2027,20 +2040,24 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
                 try {
                     int switchSegmentRecSize = RecordV1Serializer.REC_TYPE_SIZE + RecordV1Serializer.FILE_WAL_POINTER_SIZE;
+
                     if (rollOver && written < (maxSegmentSize - switchSegmentRecSize)) {
                         //it is expected there is sufficient space for this record because rollover should run early
                         final ByteBuffer buf = ByteBuffer.allocate(switchSegmentRecSize);
                         buf.put((byte)(WALRecord.RecordType.SWITCH_SEGMENT_RECORD.ordinal() + 1));
-                        final FileWALPointer pointer = new FileWALPointer(idx, (int)ch.position(), -1);
+
+                        final FileWALPointer pointer = new FileWALPointer(idx, (int)fileIO.position(), -1);
                         RecordV1Serializer.putPosition(buf, pointer);
+
                         buf.rewind();
-                        ch.write(buf, written);
+
+                        fileIO.write(buf, written);
 
                         if (mode == WALMode.DEFAULT)
-                            ch.force(false);
+                            fileIO.force();
                     }
 
-                    ch.close();
+                    fileIO.close();
                 }
                 catch (IOException e) {
                     throw new IgniteCheckedException(e);
@@ -2064,13 +2081,15 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             try {
                 WALRecord rec = head.get();
 
-                assert rec instanceof FakeRecord : "Expected head FakeRecord, actual head "
+                if (envFailed == null) {
+                    assert rec instanceof FakeRecord : "Expected head FakeRecord, actual head "
                     + (rec != null ? rec.getClass().getSimpleName() : "null");
 
-                assert written == lastFsyncPos || mode != WALMode.DEFAULT :
+                    assert written == lastFsyncPos || mode != WALMode.DEFAULT :
                     "fsync [written=" + written + ", lastFsync=" + lastFsyncPos + ']';
+                }
 
-                ch = null;
+                fileIO = null;
 
                 nextSegment.signalAll();
             }
@@ -2086,7 +2105,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             lock.lock();
 
             try {
-                while (ch != null)
+                while (fileIO != null)
                     U.await(nextSegment);
             }
             finally {
@@ -2108,7 +2127,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             lock.lock();
 
             try {
-                assert ch != null : "Writing to a closed segment.";
+                assert fileIO != null : "Writing to a closed segment.";
 
                 checkEnvironment();
 
@@ -2151,10 +2170,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 assert size > 0 : size;
 
                 try {
-                    assert written == ch.position();
+                    assert written == fileIO.position();
 
                     do {
-                        ch.write(buf);
+                        fileIO.write(buf);
                     }
                     while (buf.hasRemaining());
 
@@ -2162,7 +2181,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
                     metrics.onWalBytesWritten(size);
 
-                    assert written == ch.position();
+                    assert written == fileIO.position();
                 }
                 catch (IOException e) {
                     invalidateEnvironmentLocked(e);
@@ -2215,25 +2234,16 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
 
         /**
-         * @throws StorageException If environment is no longer valid and we missed a WAL write.
-         */
-        private void checkEnvironment() throws StorageException {
-            if (envFailed != null)
-                throw new StorageException("Failed to flush WAL buffer (environment was invalidated by a " +
-                    "previous error)", envFailed);
-        }
-
-        /**
          * @return Safely reads current position of the file channel as String. Will return "null" if channel is null.
          */
         private String safePosition() {
-            FileChannel ch = this.ch;
+            FileIO io = this.fileIO;
 
-            if (ch == null)
+            if (io == null)
                 return "null";
 
             try {
-                return String.valueOf(ch.position());
+                return String.valueOf(io.position());
             }
             catch (IOException e) {
                 return "{Failed to read channel position: " + e.getMessage() + "}";
@@ -2320,6 +2330,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             @Nullable FileWALPointer end,
             PersistentStoreConfiguration psCfg,
             @NotNull RecordSerializer serializer,
+            FileIOFactory ioFactory,
             FileArchiver archiver,
             IgniteLogger log,
             int tlbSize
@@ -2327,6 +2338,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             super(log,
                 cctx,
                 serializer,
+                ioFactory,
                 psCfg.getWalRecordIteratorBufferSize());
             this.walWorkDir = walWorkDir;
             this.walArchiveDir = walArchiveDir;
@@ -2479,11 +2491,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      */
     private void doFlush() {
         final FileWriteHandle hnd = currentHandle();
-
         try {
             hnd.flush(hnd.head.get());
         }
-        catch (IgniteCheckedException e) {
+        catch (Exception e) {
             U.warn(log, "Failed to flush WAL record queue", e);
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
index 8ea0585..4e3998b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -24,6 +24,7 @@ import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -34,15 +35,18 @@ public class IgniteWalIteratorFactory {
     private final IgniteLogger log;
     /** Page size, in standalone iterator mode this value can't be taken from memory configuration */
     private final int pageSize;
+    /** Factory to provide I/O interfaces for read/write operations with files */
+    private final FileIOFactory ioFactory;
 
     /**
      * Creates WAL files iterator factory
      * @param log Logger.
      * @param pageSize Page size, size is validated
      */
-    public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, final int pageSize) {
+    public IgniteWalIteratorFactory(@NotNull IgniteLogger log, @NotNull FileIOFactory ioFactory, int pageSize) {
         this.log = log;
         this.pageSize = pageSize;
+        this.ioFactory = ioFactory;
         new MemoryConfiguration().setPageSize(pageSize); // just for validate
     }
 
@@ -57,7 +61,7 @@ public class IgniteWalIteratorFactory {
      * @throws IgniteCheckedException if failed to read folder
      */
     public WALIterator iteratorArchiveDirectory(@NotNull final File walDirWithConsistentId) throws IgniteCheckedException {
-        return new StandaloneWalRecordsIterator(walDirWithConsistentId, log, prepareSharedCtx());
+        return new StandaloneWalRecordsIterator(walDirWithConsistentId, log, prepareSharedCtx(), ioFactory);
     }
 
     /**
@@ -69,7 +73,7 @@ public class IgniteWalIteratorFactory {
      * @throws IgniteCheckedException if failed to read files
      */
     public WALIterator iteratorArchiveFiles(@NotNull final File ...files) throws IgniteCheckedException {
-        return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), false, files);
+        return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), ioFactory, false, files);
     }
 
     /**
@@ -81,7 +85,7 @@ public class IgniteWalIteratorFactory {
      * @throws IgniteCheckedException if failed to read files
      */
     public WALIterator iteratorWorkFiles(@NotNull final File ...files) throws IgniteCheckedException {
-        return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), true, files);
+        return new StandaloneWalRecordsIterator(log, prepareSharedCtx(), ioFactory, true, files);
     }
 
     /**
@@ -93,6 +97,7 @@ public class IgniteWalIteratorFactory {
         final StandaloneIgniteCacheDatabaseSharedManager dbMgr = new StandaloneIgniteCacheDatabaseSharedManager();
 
         dbMgr.setPageSize(pageSize);
+
         return new GridCacheSharedContext<>(
             kernalCtx, null, null, null,
             null, null, dbMgr, null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index df932e6..02b9352 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridComponent;
@@ -82,13 +83,25 @@ import org.jetbrains.annotations.Nullable;
  * Dummy grid kernal context
  */
 public class StandaloneGridKernalContext implements GridKernalContext {
+    /** */
     private IgniteLogger log;
 
+    /** */
+    private IgnitePluginProcessor pluginProc;
+
     /**
      * @param log Logger.
      */
     StandaloneGridKernalContext(IgniteLogger log) {
         this.log = log;
+
+        try {
+            pluginProc = new StandaloneIgnitePluginProcessor(
+                this, config());
+        }
+        catch (IgniteCheckedException e) {
+            throw new IllegalStateException("Must not fail on empty providers list.", e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -278,7 +291,7 @@ public class StandaloneGridKernalContext implements GridKernalContext {
 
     /** {@inheritDoc} */
     @Override public IgnitePluginProcessor plugins() {
-        return null;
+        return pluginProc;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgnitePluginProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgnitePluginProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgnitePluginProcessor.java
new file mode 100644
index 0000000..838fc85
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneIgnitePluginProcessor.java
@@ -0,0 +1,38 @@
+/*
+ * 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.persistence.wal.reader;
+
+import java.util.Collections;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
+import org.apache.ignite.plugin.PluginProvider;
+
+/**
+ *
+ */
+public class StandaloneIgnitePluginProcessor extends IgnitePluginProcessor {
+    /**
+     * @param ctx Kernal context.
+     * @param cfg Ignite configuration.
+     */
+    public StandaloneIgnitePluginProcessor(GridKernalContext ctx, IgniteConfiguration cfg) throws IgniteCheckedException {
+        super(ctx, cfg, Collections.<PluginProvider>emptyList());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index f17c112..ecad70a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -21,10 +21,7 @@ import java.io.DataInput;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -33,7 +30,10 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
@@ -83,14 +83,17 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
      * @param walFilesDir Wal files directory. Should already contain node consistent ID as subfolder
      * @param log Logger.
      * @param sharedCtx Shared context.
+     * @param ioFactory File I/O factory.
      */
     StandaloneWalRecordsIterator(
-        @NotNull final File walFilesDir,
-        @NotNull final IgniteLogger log,
-        @NotNull final GridCacheSharedContext sharedCtx) throws IgniteCheckedException {
+        @NotNull File walFilesDir,
+        @NotNull IgniteLogger log,
+        @NotNull GridCacheSharedContext sharedCtx,
+        @NotNull FileIOFactory ioFactory) throws IgniteCheckedException {
         super(log,
             sharedCtx,
             new RecordV1Serializer(sharedCtx),
+            ioFactory,
             BUF_SIZE);
         init(walFilesDir, false, null);
         advance();
@@ -101,17 +104,20 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
      *
      * @param log Logger.
      * @param sharedCtx Shared context.
+     * @param ioFactory File I/O factory.
      * @param workDir Work directory is scanned, false - archive
      * @param walFiles Wal files.
      */
     StandaloneWalRecordsIterator(
-        @NotNull final IgniteLogger log,
-        @NotNull final GridCacheSharedContext sharedCtx,
-        final boolean workDir,
-        @NotNull final File... walFiles) throws IgniteCheckedException {
+            @NotNull IgniteLogger log,
+            @NotNull GridCacheSharedContext sharedCtx,
+            @NotNull FileIOFactory ioFactory,
+            boolean workDir,
+            @NotNull File... walFiles) throws IgniteCheckedException {
         super(log,
             sharedCtx,
             new RecordV1Serializer(sharedCtx),
+            ioFactory,
             BUF_SIZE);
         this.workDir = workDir;
         init(null, workDir, walFiles);
@@ -138,10 +144,12 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
         }
         else {
             this.workDir = workDir;
+
             if (workDir)
                 walFileDescriptors = scanIndexesFromFileHeaders(walFiles);
             else
                 walFileDescriptors = new ArrayList<>(Arrays.asList(FileWriteAheadLogManager.scan(walFiles)));
+
             curWalSegmIdx = !walFileDescriptors.isEmpty() ? walFileDescriptors.get(0).getIdx() : 0;
         }
         curWalSegmIdx--;
@@ -172,13 +180,10 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
 
             FileWALPointer ptr;
 
-            try (RandomAccessFile rf = new RandomAccessFile(file, "r");) {
-                final FileChannel ch = rf.getChannel();
-                final ByteBuffer buf = ByteBuffer.allocate(HEADER_RECORD_SIZE);
-
-                buf.order(ByteOrder.nativeOrder());
+            try (FileIO fileIO = ioFactory.create(file, "r")) {
+                final DataInput in = new FileInput(fileIO,
+                    new ByteBufferExpander(HEADER_RECORD_SIZE, ByteOrder.nativeOrder()));
 
-                final DataInput in = new FileInput(ch, buf);
                 // Header record must be agnostic to the serializer version.
                 final int type = in.readUnsignedByte();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
index 6847482..e086258 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsRecoveryAfterFileCorruptionTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.persistence;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
-import java.nio.channels.FileChannel;
 import java.util.Collection;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -43,7 +42,7 @@ import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl;
@@ -191,13 +190,13 @@ public class IgnitePdsRecoveryAfterFileCorruptionTest extends GridCommonAbstract
 
         FilePageStore filePageStore = (FilePageStore)store;
 
-        FileChannel ch = U.field(filePageStore, "ch");
+        FileIO fileIO = U.field(filePageStore, "fileIO");
 
-        long size = ch.size();
+        long size = fileIO.size();
 
-        ch.write(ByteBuffer.allocate((int)size - FilePageStore.HEADER_SIZE), FilePageStore.HEADER_SIZE);
+        fileIO.write(ByteBuffer.allocate((int)size - FilePageStore.HEADER_SIZE), FilePageStore.HEADER_SIZE);
 
-        ch.force(false);
+        fileIO.force();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
new file mode 100644
index 0000000..cad10ae
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
@@ -0,0 +1,195 @@
+/*
+ * 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.persistence.db.wal;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.GridKernalState;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+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;
+
+/**
+ *
+ */
+public class IgniteWalFlushFailoverTest extends GridCommonAbstractTest {
+
+    /** */
+    private static final String TEST_CACHE = "testCache";
+
+    /** */
+    private boolean flushByTimeout;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 30_000L;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration(TEST_CACHE)
+                .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration()
+                .setName("dfltMemPlc")
+                .setInitialSize(2 * 1024L * 1024L * 1024L);
+
+        MemoryConfiguration memCfg = new MemoryConfiguration()
+                .setMemoryPolicies(memPlcCfg)
+                .setDefaultMemoryPolicyName(memPlcCfg.getName());
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        PersistentStoreConfiguration storeCfg = new PersistentStoreConfiguration()
+                .setFileIOFactory(new FailingFileIOFactory())
+                .setWalMode(WALMode.BACKGROUND)
+                // Setting WAL Segment size to high values forces flushing by timeout.
+                .setWalSegmentSize(flushByTimeout ? 500_000 : 50_000);
+
+        cfg.setPersistentStoreConfiguration(storeCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test flushing error recovery when flush is triggered asynchronously by timeout
+     *
+     * @throws Exception In case of fail
+     */
+    public void testErrorOnFlushByTimeout() throws Exception {
+        flushByTimeout = true;
+        flushingErrorTest();
+    }
+
+    /**
+     * Test flushing error recovery when flush is triggered directly by transaction commit
+     *
+     * @throws Exception In case of fail
+     */
+    public void testErrorOnDirectFlush() throws Exception {
+        flushByTimeout = false;
+        flushingErrorTest();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    private void flushingErrorTest() throws Exception {
+        final IgniteEx grid = startGrid(0);
+        grid.active(true);
+
+        IgniteCache<Object, Object> cache = grid.cache(TEST_CACHE);
+
+        final int iterations = 100;
+
+        try {
+            for (int i = 0; i < iterations; i++) {
+                Transaction tx = grid.transactions().txStart(
+                        TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED);
+
+                cache.put(i, "testValue" + i);
+
+                Thread.sleep(100L);
+
+                tx.commitAsync().get();
+            }
+        }
+        catch (Exception expected) {
+            // There can be any exception. Do nothing.
+        }
+
+        // We should await successful stop of node.
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override
+            public boolean apply() {
+                return grid.context().gateway().getState() == GridKernalState.STOPPED;
+            }
+        }, getTestTimeout());
+    }
+
+    /**
+     * @throws IgniteCheckedException
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+
+    /**
+     * Create File I/O which fails after second attempt to write to File
+     */
+    private static class FailingFileIOFactory implements FileIOFactory {
+        private static final long serialVersionUID = 0L;
+
+        private final FileIOFactory delegateFactory = new RandomAccessFileIOFactory();
+
+        @Override
+        public FileIO create(File file) throws IOException {
+            return create(file, "rw");
+        }
+
+        @Override
+        public FileIO create(File file, String mode) throws IOException {
+            FileIO delegate = delegateFactory.create(file, mode);
+
+            return new FileIODecorator(delegate) {
+                int writeAttempts = 2;
+
+                @Override
+                public int write(ByteBuffer sourceBuffer) throws IOException {
+                    if (--writeAttempts == 0)
+                        throw new RuntimeException("Test exception. Unable to write to file.");
+
+                    return super.write(sourceBuffer);
+                }
+            };
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/17d881ba/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
index 303f14e..b93c74d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
@@ -23,7 +23,10 @@ import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO;
+import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExpander;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
@@ -47,9 +50,10 @@ public class IgniteDataIntegrityTests extends TestCase {
 
         randomAccessFile = new RandomAccessFile(file, "rw");
 
-        fileInput = new FileInput(randomAccessFile.getChannel(), ByteBuffer.allocate(1024));
-
-        PureJavaCrc32 pureJavaCrc32 = new PureJavaCrc32();
+        fileInput = new FileInput(
+            new RandomAccessFileIO(randomAccessFile),
+            new ByteBufferExpander(1024, ByteOrder.BIG_ENDIAN)
+        );
 
         ByteBuffer buf = ByteBuffer.allocate(1024);
         ThreadLocalRandom curr = ThreadLocalRandom.current();


[10/41] ignite git commit: Merge remote-tracking branch 'upstream/ignite-2.1.2' into ignite-2.1.2

Posted by sb...@apache.org.
Merge remote-tracking branch 'upstream/ignite-2.1.2' into ignite-2.1.2


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

Branch: refs/heads/ignite-5578-1
Commit: d232648711c882ed304eae89ae19f151067649fd
Parents: 2a5390b 7adb111
Author: devozerov <vo...@gridgain.com>
Authored: Fri Jul 7 15:02:29 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 7 15:02:29 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 84 ++++++++++++--------
 1 file changed, 50 insertions(+), 34 deletions(-)
----------------------------------------------------------------------



[29/41] ignite git commit: IGNITE-5694 - Added retries for getCollection in DataStructuresProcessor - Fixes #2240.

Posted by sb...@apache.org.
IGNITE-5694 - Added retries for getCollection in DataStructuresProcessor - Fixes #2240.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5578-1
Commit: 17694c16d449dea4d4e39a271dd39e838c581858
Parents: 36716fb
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Mon Jul 10 11:59:19 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 10 11:59:19 2017 +0300

----------------------------------------------------------------------
 .../processors/datastructures/DataStructuresProcessor.java     | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/17694c16/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 5564b79..33a2fd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1031,7 +1031,11 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
             }
         }
 
-        return c.applyx(cache.context());
+        return retryTopologySafe(new IgniteOutClosureX<T>() {
+            @Override public T applyx() throws IgniteCheckedException {
+                return c.applyx(cache.context());
+            }
+        });
     }
 
     /**


[06/41] ignite git commit: Fixed formatting.

Posted by sb...@apache.org.
Fixed formatting.


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

Branch: refs/heads/ignite-5578-1
Commit: 4fc8124d9538625cb0a95c918634b43ec1319bee
Parents: ab52671
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 14:43:28 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 14:43:28 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 40 ++++++++++----------
 1 file changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4fc8124d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 3e98b01..320ef06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -440,18 +440,16 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 U.writeLock(lock);
 
-            try {
-
-
-                if (stopping)
-                    return;
+                try {
+                    if (stopping)
+                        return;
 
-                GridDhtPartitionExchangeId exchId = exchFut.exchangeId();assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" +
-                    topVer + ", exchId=" + exchId + ']';
+                    GridDhtPartitionExchangeId exchId = exchFut.exchangeId();assert topVer.equals(exchId.topologyVersion()) : "Invalid topology version [topVer=" +
+                        topVer + ", exchId=" + exchId + ']';
 
                     if (exchId.isLeft() && exchFut.serverNodeDiscoveryEvent())
                         removeNode(exchId.nodeId());
-
+    
                     ClusterNode oldest = discoCache.oldestAliveServerNodeWithCache();
 
                     if (log.isDebugEnabled())
@@ -461,10 +459,11 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                     cntrMap.clear();
 
-                boolean grpStarted = exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom());// If this is the oldest node.
-                if (oldest != null && (loc.equals(oldest) || grpStarted)) {
-                    if (node2part == null) {
-                        node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
+                    boolean grpStarted = exchFut.cacheGroupAddedOnExchange(grp.groupId(), grp.receivedFrom());// If this is the oldest node.
+
+                    if (oldest != null && (loc.equals(oldest) || grpStarted)) {
+                        if (node2part == null) {
+                            node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 
                             if (log.isDebugEnabled())
                                 log.debug("Created brand new full topology map on oldest node [exchId=" +
@@ -486,16 +485,17 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         }
                     }
 
-                if (grpStarted ||
-                    exchFut.discoveryEvent().type() == EVT_DISCOVERY_CUSTOM_EVT ||
-                    exchFut.serverNodeDiscoveryEvent()) {if (affReady)
-                    initPartitions0(exchFut, updateSeq);
-                else {
-                    List<List<ClusterNode>> aff = grp.affinity().idealAssignment();
+                    if (grpStarted ||
+                        exchFut.discoveryEvent().type() == EVT_DISCOVERY_CUSTOM_EVT ||
+                        exchFut.serverNodeDiscoveryEvent()) {
+                        if (affReady)
+                            initPartitions0(exchFut, updateSeq);
+                        else {
+                            List<List<ClusterNode>> aff = grp.affinity().idealAssignment();
 
-                        createPartitions(aff, updateSeq);
+                            createPartitions(aff, updateSeq);
+                        }
                     }
-                }
 
                     consistencyCheck();
 


[38/41] ignite git commit: ignite-5075 Rename cacheId -> groupId

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/f2568b76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
index 665bb56..8fc2bdb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
@@ -74,7 +74,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void read(int cacheId, long pageId, ByteBuffer pageBuf) throws IgniteCheckedException {
+    @Override public void read(int grpId, long pageId, ByteBuffer pageBuf) throws IgniteCheckedException {
 
     }
 
@@ -84,35 +84,35 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void readHeader(int cacheId, int partId, ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void readHeader(int grpId, int partId, ByteBuffer buf) throws IgniteCheckedException {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void write(int cacheId, long pageId, ByteBuffer pageBuf, int tag) throws IgniteCheckedException {
+    @Override public void write(int grpId, long pageId, ByteBuffer pageBuf, int tag) throws IgniteCheckedException {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void sync(int cacheId, int partId) throws IgniteCheckedException {
+    @Override public void sync(int grpId, int partId) throws IgniteCheckedException {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void ensure(int cacheId, int partId) throws IgniteCheckedException {
+    @Override public void ensure(int grpId, int partId) throws IgniteCheckedException {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public long pageOffset(int cacheId, long pageId) throws IgniteCheckedException {
+    @Override public long pageOffset(int grpId, long pageId) throws IgniteCheckedException {
         return 0;
     }
 
     /** {@inheritDoc} */
-    @Override public long allocatePage(int cacheId, int partId, byte flags) throws IgniteCheckedException {
+    @Override public long allocatePage(int grpId, int partId, byte flags) throws IgniteCheckedException {
         long root = PageIdUtils.pageId(partId, flags, 0);
 
-        FullPageId fullId = new FullPageId(root, cacheId);
+        FullPageId fullId = new FullPageId(root, grpId);
 
         AtomicInteger allocator = allocators.get(fullId);
 
@@ -123,10 +123,10 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public int pages(int cacheId, int partId) throws IgniteCheckedException {
+    @Override public int pages(int grpId, int partId) throws IgniteCheckedException {
         long root = PageIdUtils.pageId(partId, (byte)0, 0);
 
-        FullPageId fullId = new FullPageId(root, cacheId);
+        FullPageId fullId = new FullPageId(root, grpId);
 
         AtomicInteger allocator = allocators.get(fullId);
 
@@ -137,7 +137,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public long metaPageId(int cacheId) {
+    @Override public long metaPageId(int grpId) {
         return 1;
     }
 


[23/41] ignite git commit: IGNITE-5369 Added support for _key and _val names on QueryEntity XML and code generation.

Posted by sb...@apache.org.
IGNITE-5369 Added support for _key and _val names on QueryEntity XML and code generation.


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

Branch: refs/heads/ignite-5578-1
Commit: eb37f538f55a471258e49ee2fbf0e8f234829def
Parents: e5c2ec5
Author: vsisko <vs...@gridgain.com>
Authored: Fri Jul 7 23:22:59 2017 +0700
Committer: Alexey Kuznetsov <ak...@gridgain.com>
Committed: Fri Jul 7 23:22:59 2017 +0700

----------------------------------------------------------------------
 .../internal/visor/query/VisorQueryEntity.java  | 32 +++++++++++++++++--
 modules/web-console/backend/app/mongo.js        |  2 ++
 .../generator/AbstractTransformer.js            |  4 +--
 .../generator/ConfigurationGenerator.js         | 33 +++++++++++++++-----
 .../states/configuration/domains/query.pug      |  9 ++++++
 .../frontend/controllers/domains-controller.js  |  7 +++--
 6 files changed, 73 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eb37f538/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryEntity.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryEntity.java
index 9f4dfe7..c67918a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryEntity.java
@@ -58,6 +58,12 @@ public class VisorQueryEntity extends VisorDataTransferObject {
     /** Table name. */
     private String tblName;
 
+    /** Key name. Can be used in field list to denote the key as a whole. */
+    private String keyFieldName;
+
+    /** Value name. Can be used in field list to denote the entire value. */
+    private String valFieldName;
+
     /** Fields to create group indexes for. */
     private List<VisorQueryIndex> grps;
 
@@ -108,6 +114,10 @@ public class VisorQueryEntity extends VisorDataTransferObject {
 
         for (QueryIndex qryIdx : qryIdxs)
             grps.add(new VisorQueryIndex(qryIdx));
+
+        tblName = q.getTableName();
+        keyFieldName = q.getKeyFieldName();
+        valFieldName = q.getValueFieldName();
     }
 
     /**
@@ -153,6 +163,20 @@ public class VisorQueryEntity extends VisorDataTransferObject {
     }
 
     /**
+     * @return Key name. Can be used in field list to denote the key as a whole.
+     */
+    public String getKeyFieldName() {
+        return keyFieldName;
+    }
+
+    /**
+     * @return Value name. Can be used in field list to denote the entire value.
+     */
+    public String getValueFieldName() {
+        return valFieldName;
+    }
+
+    /**
      * @return Fields to create group indexes for.
      */
     public List<VisorQueryIndex> getGroups() {
@@ -166,8 +190,10 @@ public class VisorQueryEntity extends VisorDataTransferObject {
         U.writeCollection(out, keyFields);
         IgfsUtils.writeStringMap(out, qryFlds);
         U.writeMap(out, aliases);
-        U.writeString(out, tblName);
         U.writeCollection(out, grps);
+        U.writeString(out, tblName);
+        U.writeString(out, keyFieldName);
+        U.writeString(out, valFieldName);
     }
 
     /** {@inheritDoc} */
@@ -177,8 +203,10 @@ public class VisorQueryEntity extends VisorDataTransferObject {
         keyFields = U.readList(in);
         qryFlds = IgfsUtils.readStringMap(in);
         aliases = U.readMap(in);
-        tblName = U.readString(in);
         grps = U.readList(in);
+        tblName = U.readString(in);
+        keyFieldName = U.readString(in);
+        valFieldName = U.readString(in);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/eb37f538/modules/web-console/backend/app/mongo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js
index 32796e6..aa11e0b 100644
--- a/modules/web-console/backend/app/mongo.js
+++ b/modules/web-console/backend/app/mongo.js
@@ -106,6 +106,8 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
         queryMetadata: {type: String, enum: ['Annotations', 'Configuration']},
         kind: {type: String, enum: ['query', 'store', 'both']},
         tableName: String,
+        keyFieldName: String,
+        valueFieldName: String,
         databaseSchema: String,
         databaseTable: String,
         keyType: String,

http://git-wip-us.apache.org/repos/asf/ignite/blob/eb37f538/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
index 0e4537e..4d66a64 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
@@ -304,8 +304,8 @@ export default class AbstractTransformer {
     }
 
     // Generate domain model for query group.
-    static domainModelQuery(domain) {
-        return this.toSection(this.generator.domainModelQuery(domain));
+    static domainModelQuery(domain, available) {
+        return this.toSection(this.generator.domainModelQuery(domain, available));
     }
 
     // Generate domain model for store group.

http://git-wip-us.apache.org/repos/asf/ignite/blob/eb37f538/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index f850dce..8c9b14c 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -1672,13 +1672,30 @@ export default class IgniteConfigurationGenerator {
     }
 
     // Generate domain model for query group.
-    static domainModelQuery(domain, cfg = this.domainConfigurationBean(domain)) {
+    static domainModelQuery(domain, available, cfg = this.domainConfigurationBean(domain)) {
         if (cfg.valueOf('queryMetadata') === 'Configuration') {
-            const fields = _.map(domain.fields,
-                (e) => ({name: e.name, className: javaTypes.fullClassName(e.className)}));
+            const fields = _.filter(_.map(domain.fields,
+                (e) => ({name: e.name, className: javaTypes.fullClassName(e.className)})), (field) => {
+                return field.name !== domain.keyFieldName && field.name !== domain.valueFieldName;
+            });
+
+            cfg.stringProperty('tableName');
+
+            if (available('2.0.0')) {
+                cfg.stringProperty('keyFieldName')
+                    .stringProperty('valueFieldName');
+
+                const keyFieldName = cfg.valueOf('keyFieldName');
+                const valFieldName = cfg.valueOf('valueFieldName');
+
+                if (keyFieldName)
+                    fields.push({name: keyFieldName, className: javaTypes.fullClassName(domain.keyType)});
+
+                if (valFieldName)
+                    fields.push({name: valFieldName, className: javaTypes.fullClassName(domain.valueType)});
+            }
 
-            cfg.stringProperty('tableName')
-                .mapProperty('fields', fields, 'fields', true)
+            cfg.mapProperty('fields', fields, 'fields', true)
                 .mapProperty('aliases', 'aliases');
 
             const indexes = _.map(domain.indexes, (index) =>
@@ -2131,12 +2148,12 @@ export default class IgniteConfigurationGenerator {
     }
 
     // Generate domain models configs.
-    static cacheDomains(domains, ccfg) {
+    static cacheDomains(domains, available, ccfg) {
         const qryEntities = _.reduce(domains, (acc, domain) => {
             if (_.isNil(domain.queryMetadata) || domain.queryMetadata === 'Configuration') {
                 const qryEntity = this.domainModelGeneral(domain);
 
-                this.domainModelQuery(domain, qryEntity);
+                this.domainModelQuery(domain, available, qryEntity);
 
                 acc.push(qryEntity);
             }
@@ -2160,7 +2177,7 @@ export default class IgniteConfigurationGenerator {
         this.cacheRebalance(cache, ccfg);
         this.cacheNearServer(cache, ccfg);
         this.cacheStatistics(cache, ccfg);
-        this.cacheDomains(cache.domains, ccfg);
+        this.cacheDomains(cache.domains, available, ccfg);
 
         return ccfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/eb37f538/modules/web-console/frontend/app/modules/states/configuration/domains/query.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/query.pug b/modules/web-console/frontend/app/modules/states/configuration/domains/query.pug
index b8564dc..19cb83d 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/domains/query.pug
+++ b/modules/web-console/frontend/app/modules/states/configuration/domains/query.pug
@@ -67,6 +67,15 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition)
                 div(ng-if=`${model}.queryMetadata === 'Configuration'`)
                     .settings-row
                         +text('Table name:', `${model}.tableName`, '"tableName"', 'false', 'Enter table name', 'Table name for this query entity')
+                    div(ng-if='$ctrl.available("2.0.0")')
+                        .settings-row
+                            +text('Key field name:', `${model}.keyFieldName`, '"keyFieldName"', 'false', 'Enter key field name',
+                                'Key name.<br/>' +
+                                'Can be used in field list to denote the key as a whole')
+                        .settings-row
+                            +text('Value field name:', `${model}.valueFieldName`, '"valueFieldName"', 'false', 'Enter value field name',
+                                'Value name.<br/>' +
+                                'Can be used in field list to denote the entire value')
                     .settings-row
                         +ignite-form-group(ng-model=queryFields ng-form=queryFieldsForm)
                             ignite-form-field-label(id='queryFields')

http://git-wip-us.apache.org/repos/asf/ignite/blob/eb37f538/modules/web-console/frontend/controllers/domains-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js
index 7a16c50..7c87ce0 100644
--- a/modules/web-console/frontend/controllers/domains-controller.js
+++ b/modules/web-console/frontend/controllers/domains-controller.js
@@ -18,10 +18,12 @@
 import templateUrl from 'views/configuration/domains-import.tpl.pug';
 
 // Controller for Domain model screen.
-export default ['$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteInput', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'AgentManager', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', 'IgniteActivitiesData',
-    function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Input, Loading, ModelNormalizer, UnsavedChangesGuard, agentMgr, LegacyTable, Resource, ErrorPopover, FormUtils, JavaTypes, SqlTypes, ActivitiesData) {
+export default ['$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteInput', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'AgentManager', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', 'IgniteActivitiesData', 'IgniteVersion',
+    function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Input, Loading, ModelNormalizer, UnsavedChangesGuard, agentMgr, LegacyTable, Resource, ErrorPopover, FormUtils, JavaTypes, SqlTypes, ActivitiesData, Version) {
         UnsavedChangesGuard.install($scope);
 
+        this.available = Version.available.bind(Version);
+
         const emptyDomain = {empty: true};
 
         let __original_value;
@@ -941,6 +943,7 @@ export default ['$rootScope', '$scope', '$http', '$state', '$filter', '$timeout'
                         const keyField = newDomain.keyFields[0];
 
                         newDomain.keyType = keyField.javaType;
+                        newDomain.keyFieldName = keyField.javaFieldName;
 
                         // Exclude key column from query fields.
                         newDomain.fields = _.filter(newDomain.fields, (field) => field.name !== keyField.javaFieldName);


[37/41] ignite git commit: Fixed NPE when test kernal context is used

Posted by sb...@apache.org.
Fixed NPE when test kernal context is used


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

Branch: refs/heads/ignite-5578-1
Commit: 56868102ef46e37a914593fa0f81317ccb8e4cc4
Parents: 17d881b
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jul 10 14:02:30 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 10 14:02:30 2017 +0300

----------------------------------------------------------------------
 .../testframework/junits/GridTestKernalContext.java       | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/56868102/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 3d3de22..6b39faa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.testframework.junits;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.concurrent.ExecutorService;
@@ -28,8 +29,10 @@ import org.apache.ignite.internal.GridKernalContextImpl;
 import org.apache.ignite.internal.GridKernalGatewayImpl;
 import org.apache.ignite.internal.GridLoggerProxy;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.testframework.GridTestUtils;
 
 /**
@@ -41,6 +44,13 @@ public class GridTestKernalContext extends GridKernalContextImpl {
      */
     public GridTestKernalContext(IgniteLogger log) {
         this(log, new IgniteConfiguration());
+
+        try {
+            add(new IgnitePluginProcessor(this, config(), Collections.<PluginProvider>emptyList()));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IllegalStateException("Must not fail for empty plugins list.", e);
+        }
     }
 
     /**


[18/41] ignite git commit: IGNITE-5582: Implemented Compute::Broadcast for C++

Posted by sb...@apache.org.
IGNITE-5582: Implemented Compute::Broadcast for C++

(cherry picked from commit fa974286e8f066a8d6aa57519edf5ec7761be095)


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

Branch: refs/heads/ignite-5578-1
Commit: 3c887378eb64d2d236073410070082e5699e8334
Parents: 99713fe
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Jul 7 16:52:31 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Fri Jul 7 16:52:31 2017 +0300

----------------------------------------------------------------------
 .../cpp/core-test/src/compute_test.cpp          |  91 ++++++-
 modules/platforms/cpp/core/include/Makefile.am  |   2 +
 .../cpp/core/include/ignite/compute/compute.h   |  66 +++++
 .../include/ignite/impl/compute/compute_impl.h  | 161 +++++++----
 .../ignite/impl/compute/compute_job_result.h    |  54 +++-
 .../ignite/impl/compute/compute_task_holder.h   | 204 +-------------
 .../compute/multiple_job_compute_task_holder.h  | 265 +++++++++++++++++++
 .../compute/single_job_compute_task_holder.h    | 212 +++++++++++++++
 .../platforms/cpp/core/project/vs/core.vcxproj  |   2 +
 .../cpp/core/project/vs/core.vcxproj.filters    |   6 +
 10 files changed, 811 insertions(+), 252 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core-test/src/compute_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/compute_test.cpp b/modules/platforms/cpp/core-test/src/compute_test.cpp
index 8c57ef1..1fd7670 100644
--- a/modules/platforms/cpp/core-test/src/compute_test.cpp
+++ b/modules/platforms/cpp/core-test/src/compute_test.cpp
@@ -476,7 +476,7 @@ BOOST_AUTO_TEST_CASE(IgniteRunAsyncLocalError)
     BOOST_CHECK_EXCEPTION(res.GetValue(), IgniteError, IsTestError);
 }
 
-BOOST_AUTO_TEST_CASE(IgniteRunTestRemote)
+BOOST_AUTO_TEST_CASE(IgniteRunRemote)
 {
     Ignite node2 = MakeNode("ComputeNode2");
     Compute compute = node.GetCompute();
@@ -489,7 +489,7 @@ BOOST_AUTO_TEST_CASE(IgniteRunTestRemote)
     BOOST_CHECK_EQUAL(Func3::res, "42.24");
 }
 
-BOOST_AUTO_TEST_CASE(IgniteRunTestRemoteError)
+BOOST_AUTO_TEST_CASE(IgniteRunRemoteError)
 {
     Ignite node2 = MakeNode("ComputeNode2");
     Compute compute = node.GetCompute();
@@ -509,5 +509,92 @@ BOOST_AUTO_TEST_CASE(IgniteRunTestRemoteError)
     BOOST_CHECK_EXCEPTION(res.GetValue(), IgniteError, IsTestError);
 }
 
+BOOST_AUTO_TEST_CASE(IgniteBroadcastLocalSync)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Broadcasting");;
+    std::vector<std::string> res = compute.Broadcast<std::string>(Func2(8, 5));
+
+    BOOST_CHECK_EQUAL(res.size(), 1);
+    BOOST_CHECK_EQUAL(res[0], "8.5");
+}
+
+BOOST_AUTO_TEST_CASE(IgniteBroadcastLocalAsync)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Broadcasting");;
+    Future< std::vector<std::string> > res = compute.BroadcastAsync<std::string>(Func2(312, 245));
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECKPOINT("Waiting with timeout");
+    res.WaitFor(100);
+
+    BOOST_CHECK(!res.IsReady());
+
+    std::vector<std::string> value = res.GetValue();
+
+    BOOST_CHECK_EQUAL(value.size(), 1);
+    BOOST_CHECK_EQUAL(value[0], "312.245");
+}
+
+BOOST_AUTO_TEST_CASE(IgniteBroadcastSyncLocalError)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Broadcasting");
+
+    BOOST_CHECK_EXCEPTION(compute.Broadcast(Func2(MakeTestError())), IgniteError, IsTestError);
+}
+
+BOOST_AUTO_TEST_CASE(IgniteBroadcastAsyncLocalError)
+{
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Broadcasting");
+    Future<void> res = compute.BroadcastAsync(Func2(MakeTestError()));
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECKPOINT("Waiting with timeout");
+    res.WaitFor(100);
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECK_EXCEPTION(res.GetValue(), IgniteError, IsTestError);
+}
+
+BOOST_AUTO_TEST_CASE(IgniteBroadcastRemote)
+{
+    Ignite node2 = MakeNode("ComputeNode2");
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Broadcasting");
+    std::vector<std::string> res = compute.Broadcast<std::string>(Func2(8, 5));
+
+    BOOST_CHECK_EQUAL(res.size(), 2);
+    BOOST_CHECK_EQUAL(res[0], "8.5");
+    BOOST_CHECK_EQUAL(res[1], "8.5");
+}
+
+BOOST_AUTO_TEST_CASE(IgniteBroadcastRemoteError)
+{
+    Ignite node2 = MakeNode("ComputeNode2");
+    Compute compute = node.GetCompute();
+
+    BOOST_CHECKPOINT("Broadcasting");
+    Future< std::vector<std::string> > res = compute.BroadcastAsync<std::string>(Func2(MakeTestError()));
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECKPOINT("Waiting with timeout");
+    res.WaitFor(100);
+
+    BOOST_CHECK(!res.IsReady());
+
+    BOOST_CHECK_EXCEPTION(res.GetValue(), IgniteError, IsTestError);
+}
 
 BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/Makefile.am b/modules/platforms/cpp/core/include/Makefile.am
index 50772cb..1e9369f 100644
--- a/modules/platforms/cpp/core/include/Makefile.am
+++ b/modules/platforms/cpp/core/include/Makefile.am
@@ -61,6 +61,8 @@ nobase_include_HEADERS = \
 	ignite/impl/compute/compute_job_holder.h \
 	ignite/impl/compute/compute_job_result.h \
 	ignite/impl/compute/compute_task_holder.h \
+	ignite/impl/compute/single_job_compute_task_holder.h \
+	ignite/impl/compute/multiple_job_compute_task_holder.h \
 	ignite/impl/handle_registry.h \
 	ignite/impl/ignite_binding_impl.h \
 	ignite/impl/ignite_environment.h \

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/ignite/compute/compute.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/compute/compute.h b/modules/platforms/cpp/core/include/ignite/compute/compute.h
index 75c8c85..9b4c9b9 100644
--- a/modules/platforms/cpp/core/include/ignite/compute/compute.h
+++ b/modules/platforms/cpp/core/include/ignite/compute/compute.h
@@ -157,6 +157,72 @@ namespace ignite
                 return impl.Get()->RunAsync<F>(action);
             }
 
+            /**
+             * Broadcasts provided ComputeFunc to all nodes in the cluster group.
+             *
+             * @tparam R Function return type. BinaryType should be specialized
+             *  for the type if it is not primitive.
+             * @tparam F Compute function type. Should implement ComputeFunc<R>
+             *  class.
+             * @param func Compute function to call.
+             * @return Vector containing computation results.
+             * @throw IgniteError in case of error.
+             */
+            template<typename R, typename F>
+            std::vector<R> Broadcast(const F& func)
+            {
+                return impl.Get()->BroadcastAsync<R, F>(func).GetValue();
+            }
+
+            /**
+             * Broadcasts provided ComputeFunc to all nodes in the cluster group.
+             *
+             * @tparam F Compute function type. Should implement ComputeFunc<R>
+             *  class.
+             * @param func Compute function to call.
+             * @throw IgniteError in case of error.
+             */
+            template<typename F>
+            void Broadcast(const F& func)
+            {
+                impl.Get()->BroadcastAsync<F, false>(func).GetValue();
+            }
+
+            /**
+             * Asyncronuously broadcasts provided ComputeFunc to all nodes in the
+             * cluster group.
+             *
+             * @tparam R Function return type. BinaryType should be specialized
+             *  for the type if it is not primitive.
+             * @tparam F Compute function type. Should implement ComputeFunc<R>
+             *  class.
+             * @param func Compute function to call.
+             * @return Future that can be used to access computation results once
+             *  they are ready.
+             * @throw IgniteError in case of error.
+             */
+            template<typename R, typename F>
+            Future< std::vector<R> > BroadcastAsync(const F& func)
+            {
+                return impl.Get()->BroadcastAsync<R, F>(func);
+            }
+
+            /**
+             * Asyncronuously broadcasts provided ComputeFunc to all nodes in the
+             * cluster group.
+             *
+             * @tparam F Compute function type. Should implement ComputeFunc<R>
+             *  class.
+             * @param func Compute function to call.
+             * @return Future that can be used to wait for action to complete.
+             * @throw IgniteError in case of error.
+             */
+            template<typename F>
+            Future<void> BroadcastAsync(const F& func)
+            {
+                return impl.Get()->BroadcastAsync<F, false>(func);
+            }
+
         private:
             /** Implementation. */
             common::concurrent::SharedPointer<impl::compute::ComputeImpl> impl;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
index 63f9a46..4ba1c1c 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_impl.h
@@ -26,11 +26,10 @@
 #include <ignite/common/common.h>
 #include <ignite/common/promise.h>
 #include <ignite/impl/interop/interop_target.h>
-#include <ignite/impl/compute/compute_task_holder.h>
+#include <ignite/impl/compute/single_job_compute_task_holder.h>
+#include <ignite/impl/compute/multiple_job_compute_task_holder.h>
 #include <ignite/impl/compute/cancelable_impl.h>
 
-#include <ignite/ignite_error.h>
-
 namespace ignite
 {
     namespace impl
@@ -50,7 +49,9 @@ namespace ignite
                 {
                     enum Type
                     {
-                        Unicast = 5
+                        BROADCAST = 2,
+
+                        UNICAST = 5,
                     };
                 };
 
@@ -66,41 +67,113 @@ namespace ignite
                  * Asyncronuously calls provided ComputeFunc on a node within
                  * the underlying cluster group.
                  *
-                 * @tparam F Compute function type. Should implement ComputeFunc
-                 *  class.
-                 * @tparam R Call return type. BinaryType should be specialized for
-                 *  the type if it is not primitive. Should not be void. For
+                 * @tparam F Compute function type. Should implement
+                 *  ComputeFunc<R> class.
+                 * @tparam R Call return type. BinaryType should be specialized
+                 *  for the type if it is not primitive. Should not be void. For
                  *  non-returning methods see Compute::Run().
                  * @param func Compute function to call.
-                 * @return Future that can be used to acess computation result once
-                 *  it's ready.
-                 * @throw IgniteError in case of error.
+                 * @return Future that can be used to acess computation result
+                 *  once it's ready.
                  */
                 template<typename R, typename F>
                 Future<R> CallAsync(const F& func)
                 {
-                    common::concurrent::SharedPointer<interop::InteropMemory> mem = GetEnvironment().AllocateMemory();
-                    interop::InteropOutputStream out(mem.Get());
-                    binary::BinaryWriterImpl writer(&out, GetEnvironment().GetTypeManager());
+                    typedef ComputeJobHolderImpl<F, R> JobType;
+                    typedef SingleJobComputeTaskHolder<F, R> TaskType;
+
+                    return PerformTask<R, F, JobType, TaskType>(Operation::UNICAST, func);
+                }
+
+                /**
+                 * Asyncronuously runs provided ComputeFunc on a node within
+                 * the underlying cluster group.
+                 *
+                 * @tparam F Compute action type. Should implement
+                 *  ComputeFunc<R> class.
+                 * @param action Compute action to call.
+                 * @return Future that can be used to wait for action
+                 *  to complete.
+                 */
+                template<typename F>
+                Future<void> RunAsync(const F& action)
+                {
+                    typedef ComputeJobHolderImpl<F, void> JobType;
+                    typedef SingleJobComputeTaskHolder<F, void> TaskType;
+
+                    return PerformTask<void, F, JobType, TaskType>(Operation::UNICAST, action);
+                }
+
+                /**
+                 * Asyncronuously broadcasts provided ComputeFunc to all nodes
+                 * in the underlying cluster group.
+                 *
+                 * @tparam F Compute function type. Should implement
+                 *  ComputeFunc<R> class.
+                 * @tparam R Call return type. BinaryType should be specialized
+                 *  for the type if it is not primitive. Should not be void. For
+                 *  non-returning methods see Compute::Run().
+                 * @param func Compute function to call.
+                 * @return Future that can be used to acess computation result
+                 *  once it's ready.
+                 */
+                template<typename R, typename F>
+                Future< std::vector<R> > BroadcastAsync(const F& func)
+                {
+                    typedef ComputeJobHolderImpl<F, R> JobType;
+                    typedef MultipleJobComputeTaskHolder<F, R> TaskType;
+
+                    return PerformTask<std::vector<R>, F, JobType, TaskType>(Operation::BROADCAST, func);
+                }
+
+                /**
+                 * Asyncronuously broadcasts provided ComputeFunc to all nodes
+                 * in the underlying cluster group.
+                 *
+                 * @tparam F Compute function type. Should implement
+                 *  ComputeFunc<R> class.
+                 * @param func Compute function to call.
+                 * @return Future that can be used to acess computation result
+                 *  once it's ready.
+                 */
+                template<typename F, bool>
+                Future<void> BroadcastAsync(const F& func)
+                {
+                    typedef ComputeJobHolderImpl<F, void> JobType;
+                    typedef MultipleJobComputeTaskHolder<F, void> TaskType;
+
+                    return PerformTask<void, F, JobType, TaskType>(Operation::BROADCAST, func);
+                }
 
-                    common::concurrent::SharedPointer<ComputeJobHolder> job(new ComputeJobHolderImpl<F, R>(func));
+            private:
+                /**
+                 * Perform job.
+                 *
+                 * @tparam F Compute function type. Should implement
+                 *  ComputeFunc<R> class.
+                 * @tparam R Call return type. BinaryType should be specialized
+                 *  for the type if it is not primitive.
+                 * @tparam J Job type.
+                 * @tparam T Task type.
+                 *
+                 * @param operation Operation type.
+                 * @param func Function.
+                 * @return Future that can be used to acess computation result
+                 *  once it's ready.
+                 */
+                template<typename R, typename F, typename J, typename T>
+                Future<R> PerformTask(Operation::Type operation, const F& func)
+                {
+                    common::concurrent::SharedPointer<ComputeJobHolder> job(new J(func));
 
                     int64_t jobHandle = GetEnvironment().GetHandleRegistry().Allocate(job);
 
-                    ComputeTaskHolderImpl<F, R>* taskPtr = new ComputeTaskHolderImpl<F, R>(jobHandle);
+                    T* taskPtr = new T(jobHandle);
                     common::concurrent::SharedPointer<ComputeTaskHolder> task(taskPtr);
 
                     int64_t taskHandle = GetEnvironment().GetHandleRegistry().Allocate(task);
 
-                    writer.WriteInt64(taskHandle);
-                    writer.WriteInt32(1);
-                    writer.WriteInt64(jobHandle);
-                    writer.WriteObject<F>(func);
-
-                    out.Synchronize();
-
-                    jobject target = InStreamOutObject(Operation::Unicast, *mem.Get());
-                    std::auto_ptr<common::Cancelable> cancelable(new CancelableImpl(GetEnvironmentPointer(), target));
+                    std::auto_ptr<common::Cancelable> cancelable = PerformTask(operation, jobHandle, taskHandle, func);
 
                     common::Promise<R>& promise = taskPtr->GetPromise();
                     promise.SetCancelTarget(cancelable);
@@ -109,48 +182,38 @@ namespace ignite
                 }
 
                 /**
-                 * Asyncronuously runs provided ComputeFunc on a node within
-                 * the underlying cluster group.
+                 * Perform job.
                  *
-                 * @tparam F Compute action type. Should implement ComputeAction
-                 *  class.
-                 * @param action Compute action to call.
-                 * @return Future that can be used to wait for action to complete.
-                 * @throw IgniteError in case of error.
+                 * @tparam F Compute function type. Should implement
+                 *  ComputeFunc<R> class.
+                 *
+                 * @param operation Operation type.
+                 * @param jobHandle Job Handle.
+                 * @param taskHandle Task Handle.
+                 * @param func Function.
+                 * @return Cancelable auto pointer.
                  */
                 template<typename F>
-                Future<void> RunAsync(const F& action)
+                std::auto_ptr<common::Cancelable> PerformTask(Operation::Type operation, int64_t jobHandle,
+                    int64_t taskHandle, const F& func)
                 {
                     common::concurrent::SharedPointer<interop::InteropMemory> mem = GetEnvironment().AllocateMemory();
                     interop::InteropOutputStream out(mem.Get());
                     binary::BinaryWriterImpl writer(&out, GetEnvironment().GetTypeManager());
 
-                    common::concurrent::SharedPointer<ComputeJobHolder> job(new ComputeJobHolderImpl<F, void>(action));
-
-                    int64_t jobHandle = GetEnvironment().GetHandleRegistry().Allocate(job);
-
-                    ComputeTaskHolderImpl<F, void>* taskPtr = new ComputeTaskHolderImpl<F, void>(jobHandle);
-                    common::concurrent::SharedPointer<ComputeTaskHolder> task(taskPtr);
-
-                    int64_t taskHandle = GetEnvironment().GetHandleRegistry().Allocate(task);
-
                     writer.WriteInt64(taskHandle);
                     writer.WriteInt32(1);
                     writer.WriteInt64(jobHandle);
-                    writer.WriteObject<F>(action);
+                    writer.WriteObject<F>(func);
 
                     out.Synchronize();
 
-                    jobject target = InStreamOutObject(Operation::Unicast, *mem.Get());
+                    jobject target = InStreamOutObject(operation, *mem.Get());
                     std::auto_ptr<common::Cancelable> cancelable(new CancelableImpl(GetEnvironmentPointer(), target));
 
-                    common::Promise<void>& promise = taskPtr->GetPromise();
-                    promise.SetCancelTarget(cancelable);
-
-                    return promise.GetFuture();
+                    return cancelable;
                 }
 
-            private:
                 IGNITE_NO_COPY_ASSIGNMENT(ComputeImpl);
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
index 0874522..9d3dfea 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_job_result.h
@@ -36,6 +36,28 @@ namespace ignite
     {
         namespace compute
         {
+            struct ComputeJobResultPolicy
+            {
+                enum Type
+                {
+                    /**
+                    * Wait for results if any are still expected. If all results have been received -
+                    * it will start reducing results.
+                    */
+                    WAIT = 0,
+
+                    /**
+                    * Ignore all not yet received results and start reducing results.
+                    */
+                    REDUCE = 1,
+
+                    /**
+                    * Fail-over job to execute on another node.
+                    */
+                    FAILOVER = 2
+                };
+            };
+
             /**
              * Used to hold compute job result.
              */
@@ -65,16 +87,36 @@ namespace ignite
                 }
 
                 /**
+                 * Get result value.
+                 *
+                 * @return Result.
+                 */
+                const ResultType& GetResult() const
+                {
+                    return res;
+                }
+
+                /**
                  * Set error.
                  *
                  * @param error Error to set.
                  */
-                void SetError(const IgniteError error)
+                void SetError(const IgniteError& error)
                 {
                     err = error;
                 }
 
                 /**
+                 * Get error.
+                 *
+                 * @return Error.
+                 */
+                const IgniteError& GetError() const
+                {
+                    return err;
+                }
+
+                /**
                  * Set promise to a state which corresponds to result.
                  *
                  * @param promise Promise, which state to set.
@@ -192,6 +234,16 @@ namespace ignite
                 }
 
                 /**
+                 * Get error.
+                 *
+                 * @return Error.
+                 */
+                const IgniteError& GetError() const
+                {
+                    return err;
+                }
+
+                /**
                  * Set promise to a state which corresponds to result.
                  *
                  * @param promise Promise, which state to set.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
index f627f27..66276d1 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/compute_task_holder.h
@@ -17,17 +17,14 @@
 
 /**
  * @file
- * Declares ignite::impl::compute::ComputeTaskHolder class and
- * ignite::impl::compute::ComputeTaskHolderImpl class template.
+ * Declares ignite::impl::compute::ComputeTaskHolder.
  */
 
-#ifndef _IGNITE_IMPL_COMPUTE_COMPUTE_TASK_IMPL
-#define _IGNITE_IMPL_COMPUTE_COMPUTE_TASK_IMPL
+#ifndef _IGNITE_IMPL_COMPUTE_COMPUTE_TASK_HOLDER
+#define _IGNITE_IMPL_COMPUTE_COMPUTE_TASK_HOLDER
 
 #include <stdint.h>
 
-#include <ignite/common/promise.h>
-#include <ignite/impl/compute/compute_job_result.h>
 #include <ignite/impl/compute/compute_job_holder.h>
 
 namespace ignite
@@ -36,28 +33,6 @@ namespace ignite
     {
         namespace compute
         {
-            struct ComputeJobResultPolicy
-            {
-                enum Type
-                {
-                    /**
-                     * Wait for results if any are still expected. If all results have been received -
-                     * it will start reducing results.
-                     */
-                    WAIT = 0,
-
-                    /**
-                     * Ignore all not yet received results and start reducing results.
-                     */
-                    REDUCE = 1,
-
-                    /**
-                     * Fail-over job to execute on another node.
-                     */
-                    FAILOVER = 2
-                };
-            };
-
             /**
              * Compute task holder. Internal helper class.
              * Used to handle tasks in general way, without specific types.
@@ -120,179 +95,8 @@ namespace ignite
                 /** Related job handle. */
                 int64_t handle;
             };
-
-            /**
-             * Compute task holder type-specific implementation.
-             */
-            template<typename F, typename R>
-            class ComputeTaskHolderImpl : public ComputeTaskHolder
-            {
-            public:
-                typedef F JobType;
-                typedef R ResultType;
-
-                /**
-                 * Constructor.
-                 *
-                 * @param handle Job handle.
-                 */
-                ComputeTaskHolderImpl(int64_t handle) :
-                    ComputeTaskHolder(handle)
-                {
-                    // No-op.
-                }
-
-                /**
-                 * Destructor.
-                 */
-                virtual ~ComputeTaskHolderImpl()
-                {
-                    // No-op.
-                }
-
-                /**
-                 * Process local job result.
-                 *
-                 * @param job Job.
-                 * @return Policy.
-                 */
-                virtual int32_t JobResultLocal(ComputeJobHolder& job)
-                {
-                    typedef ComputeJobHolderImpl<JobType, ResultType> ActualComputeJobHolder;
-
-                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
-
-                    res = job0.GetResult();
-
-                    return ComputeJobResultPolicy::WAIT;
-                }
-
-                /**
-                 * Process remote job result.
-                 *
-                 * @param job Job.
-                 * @param reader Reader for stream with result.
-                 * @return Policy.
-                 */
-                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
-                {
-                    res.Read(reader);
-
-                    return ComputeJobResultPolicy::WAIT;
-                }
-
-                /**
-                 * Reduce results of related jobs.
-                 */
-                virtual void Reduce()
-                {
-                    res.SetPromise(promise);
-                }
-
-                /**
-                 * Get result promise.
-                 *
-                 * @return Reference to result promise.
-                 */
-                common::Promise<ResultType>& GetPromise()
-                {
-                    return promise;
-                }
-
-            private:
-                /** Result. */
-                ComputeJobResult<ResultType> res;
-
-                /** Task result promise. */
-                common::Promise<ResultType> promise;
-            };
-
-            /**
-             * Compute task holder type-specific implementation.
-             */
-            template<typename F>
-            class ComputeTaskHolderImpl<F, void> : public ComputeTaskHolder
-            {
-            public:
-                typedef F JobType;
-
-                /**
-                 * Constructor.
-                 *
-                 * @param handle Job handle.
-                 */
-                ComputeTaskHolderImpl(int64_t handle) :
-                    ComputeTaskHolder(handle)
-                {
-                    // No-op.
-                }
-
-                /**
-                 * Destructor.
-                 */
-                virtual ~ComputeTaskHolderImpl()
-                {
-                    // No-op.
-                }
-
-                /**
-                 * Process local job result.
-                 *
-                 * @param job Job.
-                 * @return Policy.
-                 */
-                virtual int32_t JobResultLocal(ComputeJobHolder& job)
-                {
-                    typedef ComputeJobHolderImpl<JobType, void> ActualComputeJobHolder;
-
-                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
-
-                    res = job0.GetResult();
-
-                    return ComputeJobResultPolicy::WAIT;
-                }
-
-                /**
-                 * Process remote job result.
-                 *
-                 * @param job Job.
-                 * @param reader Reader for stream with result.
-                 * @return Policy.
-                 */
-                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
-                {
-                    res.Read(reader);
-
-                    return ComputeJobResultPolicy::WAIT;
-                }
-
-                /**
-                 * Reduce results of related jobs.
-                 */
-                virtual void Reduce()
-                {
-                    res.SetPromise(promise);
-                }
-
-                /**
-                 * Get result promise.
-                 *
-                 * @return Reference to result promise.
-                 */
-                common::Promise<void>& GetPromise()
-                {
-                    return promise;
-                }
-
-            private:
-                /** Result. */
-                ComputeJobResult<void> res;
-
-                /** Task result promise. */
-                common::Promise<void> promise;
-            };
         }
     }
 }
 
-#endif //_IGNITE_IMPL_COMPUTE_COMPUTE_TASK_IMPL
+#endif //_IGNITE_IMPL_COMPUTE_COMPUTE_TASK_HOLDER

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/ignite/impl/compute/multiple_job_compute_task_holder.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/multiple_job_compute_task_holder.h b/modules/platforms/cpp/core/include/ignite/impl/compute/multiple_job_compute_task_holder.h
new file mode 100644
index 0000000..9fb13f1
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/multiple_job_compute_task_holder.h
@@ -0,0 +1,265 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::MultipleJobComputeTaskHolder class template.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_MULTIPLE_JOB_COMPUTE_TASK
+#define _IGNITE_IMPL_COMPUTE_MULTIPLE_JOB_COMPUTE_TASK
+
+#include <stdint.h>
+#include <vector>
+
+#include <ignite/common/promise.h>
+#include <ignite/impl/compute/compute_job_result.h>
+#include <ignite/impl/compute/compute_task_holder.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            /**
+             * Multiple Job Compute task holder type-specific implementation.
+             * Used for broadcast.
+             *
+             * @tparam F Function type.
+             * @tparam R Function result type.
+             */
+            template<typename F, typename R>
+            class MultipleJobComputeTaskHolder : public ComputeTaskHolder
+            {
+            public:
+                typedef F JobType;
+                typedef R ResultType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param handle Job handle.
+                 */
+                MultipleJobComputeTaskHolder(int64_t handle) :
+                    ComputeTaskHolder(handle),
+                    result(new std::vector<ResultType>()),
+                    error(),
+                    promise()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~MultipleJobComputeTaskHolder()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Process local job result.
+                 *
+                 * @param job Job.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultLocal(ComputeJobHolder& job)
+                {
+                    typedef ComputeJobHolderImpl<JobType, ResultType> ActualComputeJobHolder;
+
+                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
+
+                    ProcessResult(job0.GetResult());
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Process remote job result.
+                 *
+                 * @param job Job.
+                 * @param reader Reader for stream with result.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
+                {
+                    ComputeJobResult<ResultType> res;
+
+                    res.Read(reader);
+
+                    ProcessResult(res);
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Reduce results of related jobs.
+                 */
+                virtual void Reduce()
+                {
+                    if (error.GetCode() == IgniteError::IGNITE_SUCCESS)
+                        promise.SetValue(result);
+                    else
+                        promise.SetError(error);
+                }
+
+                /**
+                 * Get result promise.
+                 *
+                 * @return Reference to result promise.
+                 */
+                common::Promise< std::vector<ResultType> >& GetPromise()
+                {
+                    return promise;
+                }
+
+            private:
+                /**
+                 * Process result.
+                 *
+                 * @param res Result.
+                 */
+                void ProcessResult(const ComputeJobResult<ResultType>& res)
+                {
+                    const IgniteError& err = res.GetError();
+
+                    if (err.GetCode() == IgniteError::IGNITE_SUCCESS)
+                        result->push_back(res.GetResult());
+                    else
+                        error = err;
+                }
+
+                /** Result. */
+                std::auto_ptr< std::vector<ResultType> > result;
+
+                /** Error. */
+                IgniteError error;
+
+                /** Task result promise. */
+                common::Promise< std::vector<ResultType> > promise;
+            };
+
+            /**
+             * Compute task holder type-specific implementation.
+             */
+            template<typename F>
+            class MultipleJobComputeTaskHolder<F, void> : public ComputeTaskHolder
+            {
+            public:
+                typedef F JobType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param handle Job handle.
+                 */
+                MultipleJobComputeTaskHolder(int64_t handle) :
+                    ComputeTaskHolder(handle)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~MultipleJobComputeTaskHolder()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Process local job result.
+                 *
+                 * @param job Job.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultLocal(ComputeJobHolder& job)
+                {
+                    typedef ComputeJobHolderImpl<JobType, void> ActualComputeJobHolder;
+
+                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
+
+                    ProcessResult(job0.GetResult());
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Process remote job result.
+                 *
+                 * @param job Job.
+                 * @param reader Reader for stream with result.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
+                {
+                    ComputeJobResult<void> res;
+
+                    res.Read(reader);
+
+                    ProcessResult(res);
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Reduce results of related jobs.
+                 */
+                virtual void Reduce()
+                {
+                    if (error.GetCode() == IgniteError::IGNITE_SUCCESS)
+                        promise.SetValue();
+                    else
+                        promise.SetError(error);
+                }
+
+                /**
+                 * Get result promise.
+                 *
+                 * @return Reference to result promise.
+                 */
+                common::Promise<void>& GetPromise()
+                {
+                    return promise;
+                }
+
+            private:
+                /**
+                 * Process result.
+                 *
+                 * @param res Result.
+                 */
+                void ProcessResult(const ComputeJobResult<void>& res)
+                {
+                    const IgniteError& err = res.GetError();
+
+                    if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+                        error = err;
+                }
+
+                /** Error. */
+                IgniteError error;
+
+                /** Task result promise. */
+                common::Promise<void> promise;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_MULTIPLE_JOB_COMPUTE_TASK

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/include/ignite/impl/compute/single_job_compute_task_holder.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/compute/single_job_compute_task_holder.h b/modules/platforms/cpp/core/include/ignite/impl/compute/single_job_compute_task_holder.h
new file mode 100644
index 0000000..9b0506a
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/compute/single_job_compute_task_holder.h
@@ -0,0 +1,212 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::compute::SingleJobComputeTaskHolder class template.
+ */
+
+#ifndef _IGNITE_IMPL_COMPUTE_SINGLE_JOB_COMPUTE_TASK
+#define _IGNITE_IMPL_COMPUTE_SINGLE_JOB_COMPUTE_TASK
+
+#include <stdint.h>
+
+#include <ignite/common/promise.h>
+#include <ignite/impl/compute/compute_job_result.h>
+#include <ignite/impl/compute/compute_task_holder.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace compute
+        {
+            /**
+             * Compute task holder type-specific implementation.
+             */
+            template<typename F, typename R>
+            class SingleJobComputeTaskHolder : public ComputeTaskHolder
+            {
+            public:
+                typedef F JobType;
+                typedef R ResultType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param handle Job handle.
+                 */
+                SingleJobComputeTaskHolder(int64_t handle) :
+                    ComputeTaskHolder(handle)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~SingleJobComputeTaskHolder()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Process local job result.
+                 *
+                 * @param job Job.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultLocal(ComputeJobHolder& job)
+                {
+                    typedef ComputeJobHolderImpl<JobType, ResultType> ActualComputeJobHolder;
+
+                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
+
+                    res = job0.GetResult();
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Process remote job result.
+                 *
+                 * @param job Job.
+                 * @param reader Reader for stream with result.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
+                {
+                    res.Read(reader);
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Reduce results of related jobs.
+                 */
+                virtual void Reduce()
+                {
+                    res.SetPromise(promise);
+                }
+
+                /**
+                 * Get result promise.
+                 *
+                 * @return Reference to result promise.
+                 */
+                common::Promise<ResultType>& GetPromise()
+                {
+                    return promise;
+                }
+
+            private:
+                /** Result. */
+                ComputeJobResult<ResultType> res;
+
+                /** Task result promise. */
+                common::Promise<ResultType> promise;
+            };
+
+            /**
+             * Compute task holder type-specific implementation.
+             */
+            template<typename F>
+            class SingleJobComputeTaskHolder<F, void> : public ComputeTaskHolder
+            {
+            public:
+                typedef F JobType;
+
+                /**
+                 * Constructor.
+                 *
+                 * @param handle Job handle.
+                 */
+                SingleJobComputeTaskHolder(int64_t handle) :
+                    ComputeTaskHolder(handle)
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~SingleJobComputeTaskHolder()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Process local job result.
+                 *
+                 * @param job Job.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultLocal(ComputeJobHolder& job)
+                {
+                    typedef ComputeJobHolderImpl<JobType, void> ActualComputeJobHolder;
+
+                    ActualComputeJobHolder& job0 = static_cast<ActualComputeJobHolder&>(job);
+
+                    res = job0.GetResult();
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Process remote job result.
+                 *
+                 * @param job Job.
+                 * @param reader Reader for stream with result.
+                 * @return Policy.
+                 */
+                virtual int32_t JobResultRemote(ComputeJobHolder& job, binary::BinaryReaderImpl& reader)
+                {
+                    res.Read(reader);
+
+                    return ComputeJobResultPolicy::WAIT;
+                }
+
+                /**
+                 * Reduce results of related jobs.
+                 */
+                virtual void Reduce()
+                {
+                    res.SetPromise(promise);
+                }
+
+                /**
+                 * Get result promise.
+                 *
+                 * @return Reference to result promise.
+                 */
+                common::Promise<void>& GetPromise()
+                {
+                    return promise;
+                }
+
+            private:
+                /** Result. */
+                ComputeJobResult<void> res;
+
+                /** Task result promise. */
+                common::Promise<void> promise;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_COMPUTE_SINGLE_JOB_COMPUTE_TASK

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/project/vs/core.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj b/modules/platforms/cpp/core/project/vs/core.vcxproj
index 9911ffe..3c3489c 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj
@@ -232,6 +232,8 @@
     <ClInclude Include="..\..\include\ignite\impl\compute\compute_job_holder.h" />
     <ClInclude Include="..\..\include\ignite\impl\compute\compute_job_result.h" />
     <ClInclude Include="..\..\include\ignite\impl\compute\compute_task_holder.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\multiple_job_compute_task_holder.h" />
+    <ClInclude Include="..\..\include\ignite\impl\compute\single_job_compute_task_holder.h" />
     <ClInclude Include="..\..\include\ignite\impl\helpers.h" />
     <ClInclude Include="..\..\include\ignite\impl\ignite_environment.h" />
     <ClInclude Include="..\..\include\ignite\impl\ignite_impl.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c887378/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
index 7b84494..27f3944 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
@@ -237,6 +237,12 @@
     <ClInclude Include="..\..\include\ignite\impl\compute\compute_task_holder.h">
       <Filter>Code\impl\compute</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\single_job_compute_task_holder.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\compute\multiple_job_compute_task_holder.h">
+      <Filter>Code\impl\compute</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <Filter Include="Code">


[05/41] ignite git commit: IGNITE-5483: Added limited support for Java8 LocalDateTime. This closes #2248.

Posted by sb...@apache.org.
IGNITE-5483: Added limited support for Java8 LocalDateTime. This closes #2248.


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

Branch: refs/heads/ignite-5578-1
Commit: 99fd75dd7766318af88a38b5640a648bba3ccdf1
Parents: 30922ed
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Fri Jul 7 14:38:36 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 7 14:38:36 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java         | 19 +++++++++++++++++++
 .../processors/query/h2/H2DatabaseType.java      | 10 ++++++++++
 .../processors/query/h2/H2RowDescriptor.java     | 13 +++++++++++++
 3 files changed, 42 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/99fd75dd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 98d123f..0c1dbf9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.query.h2;
 import java.lang.reflect.Array;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
@@ -79,8 +81,13 @@ import org.h2.command.dml.Insert;
 import org.h2.command.dml.Merge;
 import org.h2.command.dml.Update;
 import org.h2.table.Column;
+import org.h2.util.DateTimeUtils;
+import org.h2.util.LocalDateTimeUtils;
 import org.h2.value.DataType;
 import org.h2.value.Value;
+import org.h2.value.ValueDate;
+import org.h2.value.ValueTime;
+import org.h2.value.ValueTimestamp;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -711,6 +718,18 @@ public class DmlStatementsProcessor {
             return U.unmarshal(desc.context().marshaller(), (byte[]) val,
                 U.resolveClassLoader(desc.context().gridConfig()));
 
+        if (LocalDateTimeUtils.isJava8DateApiPresent()) {
+            if (val instanceof Timestamp && LocalDateTimeUtils.isLocalDateTime(expCls))
+                return LocalDateTimeUtils.valueToLocalDateTime(ValueTimestamp.get((Timestamp)val));
+
+            if (val instanceof Date && LocalDateTimeUtils.isLocalDate(expCls))
+                return LocalDateTimeUtils.valueToLocalDate(ValueDate.fromDateValue(
+                    DateTimeUtils.dateValueFromDate(((Date)val).getTime())));
+
+            if (val instanceof Time && LocalDateTimeUtils.isLocalTime(expCls))
+                return LocalDateTimeUtils.valueToLocalTime(ValueTime.get((Time)val));
+        }
+
         // We have to convert arrays of reference types manually - see https://issues.apache.org/jira/browse/IGNITE-4327
         // Still, we only can convert from Object[] to something more precise.
         if (type == Value.ARRAY && currCls != expCls) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/99fd75dd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
index 47c7eb9..8e4e639 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2DatabaseType.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2;
 
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.h2.util.LocalDateTimeUtils;
 import org.h2.value.DataType;
 
 import java.math.BigDecimal;
@@ -142,6 +143,15 @@ public enum H2DatabaseType {
         if (DataType.isGeometryClass(cls))
             return GEOMETRY;
 
+        if (LocalDateTimeUtils.isJava8DateApiPresent()) {
+            if (LocalDateTimeUtils.isLocalDate(cls))
+                return DATE;
+            else if (LocalDateTimeUtils.isLocalTime(cls))
+                return TIME;
+            else if (LocalDateTimeUtils.isLocalDateTime(cls))
+                return TIMESTAMP;
+        }
+
         return cls.isArray() && !cls.getComponentType().isPrimitive() ? ARRAY : OTHER;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/99fd75dd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
index a9bbd23..dab83d1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2RowDescriptor.java
@@ -36,6 +36,7 @@ import org.h2.message.DbException;
 import org.h2.mvstore.cache.CacheLongKeyLIRS;
 import org.h2.result.SearchRow;
 import org.h2.result.SimpleRow;
+import org.h2.util.LocalDateTimeUtils;
 import org.h2.value.DataType;
 import org.h2.value.Value;
 import org.h2.value.ValueArray;
@@ -246,14 +247,26 @@ public class H2RowDescriptor implements GridH2RowDescriptor {
                 UUID uuid = (UUID)obj;
                 return ValueUuid.get(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits());
             case Value.DATE:
+                if (LocalDateTimeUtils.isLocalDate(obj.getClass()))
+                    return LocalDateTimeUtils.localDateToDateValue(obj);
+
                 return ValueDate.get((Date)obj);
+
             case Value.TIME:
+                if (LocalDateTimeUtils.isLocalTime(obj.getClass()))
+                    return LocalDateTimeUtils.localTimeToTimeValue(obj);
+
                 return ValueTime.get((Time)obj);
+
             case Value.TIMESTAMP:
                 if (obj instanceof java.util.Date && !(obj instanceof Timestamp))
                     obj = new Timestamp(((java.util.Date)obj).getTime());
 
+                if (LocalDateTimeUtils.isLocalDateTime(obj.getClass()))
+                    return LocalDateTimeUtils.localDateTimeToValue(obj);
+
                 return ValueTimestamp.get((Timestamp)obj);
+
             case Value.DECIMAL:
                 return ValueDecimal.get((BigDecimal)obj);
             case Value.STRING:


[31/41] ignite git commit: Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2

Posted by sb...@apache.org.
Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java


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

Branch: refs/heads/ignite-5578-1
Commit: 9790a46c59e2fdc29703246db3dc7ba9abcd108a
Parents: 0b9527e 17694c1
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 12:02:41 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 12:02:41 2017 +0300

----------------------------------------------------------------------
 ...ishSnapshotOperationAckDiscoveryMessage.java |  84 -------
 .../snapshot/SnapshotCheckParameters.java       |  75 ------
 .../pagemem/snapshot/SnapshotOperation.java     | 235 -------------------
 .../pagemem/snapshot/SnapshotOperationType.java |  49 ----
 ...artSnapshotOperationAckDiscoveryMessage.java | 149 ------------
 .../StartSnapshotOperationDiscoveryMessage.java | 192 ---------------
 .../GridCachePartitionExchangeManager.java      |   9 +-
 .../processors/cache/GridCacheProcessor.java    |  22 +-
 .../cache/GridCacheSharedContext.java           |   6 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   8 +-
 .../GridDhtPartitionsExchangeFuture.java        |  54 ++---
 .../GridCacheDatabaseSharedManager.java         |   3 +-
 .../persistence/IgniteCacheSnapshotManager.java | 150 ------------
 .../persistence/file/FilePageStoreManager.java  |   2 +-
 .../snapshot/IgniteCacheSnapshotManager.java    | 161 +++++++++++++
 .../snapshot/SnapshotDiscoveryMessage.java      |  33 +++
 .../persistence/snapshot/SnapshotOperation.java |  44 ++++
 .../datastructures/DataStructuresProcessor.java |   6 +-
 .../resources/META-INF/classnames.properties    |   5 -
 .../loadtests/hashmap/GridCacheTestContext.java |   6 +-
 20 files changed, 286 insertions(+), 1007 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9790a46c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 7c62cca,a69872f..fe0a0c6
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@@ -1740,11 -1740,11 +1740,11 @@@ public class GridDhtPartitionTopologyIm
  
                          result.add(e.getKey());
                      }
-                 }
  
-                 U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
-                     "[nodeId=" + ctx.localNodeId() + ", cacheOrGroupName=" + grp.cacheOrGroupName() +
-                     ", partId=" + p + ", haveHistory=" + haveHistory + "]");
+                     U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
 -                        "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
++                        "[nodeId=" + ctx.localNodeId() + ", cacheOrGroupName=" + grp.cacheOrGroupName() +
+                         ", partId=" + p + ", haveHistory=" + haveHistory + "]");
+                 }
              }
  
              if (updateSeq)


[15/41] ignite git commit: Removed unused method GridDhtPartitionTopology.checkEvictions().

Posted by sb...@apache.org.
Removed unused method GridDhtPartitionTopology.checkEvictions().


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

Branch: refs/heads/ignite-5578-1
Commit: a1cac1b7cc1c3afdcc12afb08bf1ecdcb6f28828
Parents: ea4420e
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 15:55:18 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 15:55:18 2017 +0300

----------------------------------------------------------------------
 .../dht/GridClientPartitionTopology.java        |  5 ---
 .../dht/GridDhtPartitionTopology.java           |  7 ++--
 .../dht/GridDhtPartitionTopologyImpl.java       | 36 --------------------
 3 files changed, 2 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a1cac1b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 9e9f4fb..e751961 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -828,11 +828,6 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
         return Collections.emptyList();
     }
 
-    /** {@inheritDoc} */
-    @Override public void checkEvictions() {
-        // No-op.
-    }
-
     /**
      * Updates value for single partition.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1cac1b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index bf4e844..5f76d12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -280,11 +280,6 @@ public interface GridDhtPartitionTopology {
     public Collection<Integer> lostPartitions();
 
     /**
-     *
-     */
-    public void checkEvictions();
-
-    /**
      * @param skipZeros If {@code true} then filters out zero counters.
      * @return Partition update counters.
      */
@@ -324,6 +319,7 @@ public interface GridDhtPartitionTopology {
     /**
      * Make nodes from provided set owners for a given partition.
      * State of all current owners that aren't contained in the set will be reset to MOVING.
+     *
      * @param p Partition ID.
      * @param updateSeq If should increment sequence when updated.
      * @param owners Set of new owners.
@@ -333,6 +329,7 @@ public interface GridDhtPartitionTopology {
 
     /**
      * Callback on exchange done.
+     *
      * @param assignment New affinity assignment.
      */
     public void onExchangeDone(AffinityAssignment assignment);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1cac1b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index f49dccf..cf0dd5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1745,42 +1745,6 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /**
      * @param updateSeq Update sequence.
-     * @return {@code True} if state changed.
-     */
-    private boolean checkEvictions(long updateSeq) {
-        AffinityTopologyVersion affVer = grp.affinity().lastVersion();
-
-        boolean changed = false;
-
-        if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) {
-            List<List<ClusterNode>> aff = grp.affinity().assignments(topVer);
-
-            changed = checkEvictions(updateSeq, aff);
-
-            updateRebalanceVersion(aff);
-        }
-
-        return changed;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void checkEvictions() {
-        lock.writeLock().lock();
-
-        try {
-            long updateSeq = this.updateSeq.incrementAndGet();
-
-            node2part.newUpdateSequence(updateSeq);
-
-            checkEvictions(updateSeq);
-        }
-        finally {
-            lock.writeLock().unlock();
-        }
-    }
-
-    /**
-     * @param updateSeq Update sequence.
      * @param aff Affinity assignments.
      * @return Checks if any of the local partitions need to be evicted.
      */


[22/41] ignite git commit: ignite-2.1 Minor.

Posted by sb...@apache.org.
ignite-2.1 Minor.


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

Branch: refs/heads/ignite-5578-1
Commit: e5c2ec5a62c0d5fc268275ea9277f6933a582966
Parents: 993f7fb
Author: Alexey Kuznetsov <ak...@gridgain.com>
Authored: Fri Jul 7 23:06:52 2017 +0700
Committer: Alexey Kuznetsov <ak...@gridgain.com>
Committed: Fri Jul 7 23:06:52 2017 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCacheMetrics.java  | 12 ++----------
 .../visor/node/VisorMemoryPolicyConfiguration.java      | 11 ++---------
 .../src/main/resources/META-INF/classnames.properties   | 11 +++++++++--
 3 files changed, 13 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e5c2ec5a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
----------------------------------------------------------------------
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 d99e0c9..5d8bc81 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
@@ -639,11 +639,6 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
     }
 
     /** {@inheritDoc} */
-    @Override public byte getProtocolVersion() {
-        return V2;
-    }
-
-    /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
         U.writeEnum(out, mode);
@@ -690,6 +685,7 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
         out.writeLong(heapEntriesCnt);
         out.writeLong(offHeapAllocatedSize);
         out.writeLong(offHeapEntriesCnt);
+        out.writeLong(offHeapPrimaryEntriesCnt);
 
         out.writeInt(totalPartsCnt);
         out.writeInt(rebalancingPartsCnt);
@@ -698,8 +694,6 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
         out.writeLong(rebalancingBytesRate);
 
         out.writeObject(qryMetrics);
-
-        out.writeLong(offHeapPrimaryEntriesCnt);
     }
 
     /** {@inheritDoc} */
@@ -748,6 +742,7 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
         heapEntriesCnt = in.readLong();
         offHeapAllocatedSize = in.readLong();
         offHeapEntriesCnt = in.readLong();
+        offHeapPrimaryEntriesCnt = in.readLong();
 
         totalPartsCnt = in.readInt();
         rebalancingPartsCnt = in.readInt();
@@ -756,9 +751,6 @@ public class VisorCacheMetrics extends VisorDataTransferObject {
         rebalancingBytesRate = in.readLong();
 
         qryMetrics = (VisorQueryMetrics)in.readObject();
-
-        if (protoVer >= V2)
-            offHeapPrimaryEntriesCnt = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e5c2ec5a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
index d3153a6..bed4c4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
@@ -131,32 +131,25 @@ public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject {
     }
 
     /** {@inheritDoc} */
-    @Override public byte getProtocolVersion() {
-        return V2;
-    }
-
-    /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
+        out.writeLong(initSize);
         out.writeLong(maxSize);
         U.writeString(out, swapFilePath);
         U.writeEnum(out, pageEvictionMode);
         out.writeDouble(evictionThreshold);
         out.writeInt(emptyPagesPoolSize);
-        out.writeLong(initSize);
     }
 
     /** {@inheritDoc} */
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         name = U.readString(in);
+        initSize = in.readLong();
         maxSize = in.readLong();
         swapFilePath = U.readString(in);
         pageEvictionMode = DataPageEvictionMode.fromOrdinal(in.readByte());
         evictionThreshold = in.readDouble();
         emptyPagesPoolSize = in.readInt();
-
-        if (protoVer >= V2)
-            initSize = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e5c2ec5a/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 1528040..e34a772 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -155,6 +155,7 @@ org.apache.ignite.events.EventAdapter
 org.apache.ignite.events.IgfsEvent
 org.apache.ignite.events.JobEvent
 org.apache.ignite.events.TaskEvent
+org.apache.ignite.events.WalSegmentArchivedEvent
 org.apache.ignite.hadoop.HadoopInputSplit
 org.apache.ignite.hadoop.HadoopMapReducePlan
 org.apache.ignite.igfs.IgfsConcurrentModificationException
@@ -337,6 +338,7 @@ org.apache.ignite.internal.marshaller.optimized.OptimizedFieldType
 org.apache.ignite.internal.mem.IgniteOutOfMemoryException
 org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl$Segment
 org.apache.ignite.internal.pagemem.snapshot.FinishSnapshotOperationAckDiscoveryMessage
+org.apache.ignite.internal.pagemem.snapshot.SnapshotCheckParameters
 org.apache.ignite.internal.pagemem.snapshot.SnapshotOperation
 org.apache.ignite.internal.pagemem.snapshot.SnapshotOperationType
 org.apache.ignite.internal.pagemem.snapshot.StartSnapshotOperationAckDiscoveryMessage
@@ -407,11 +409,9 @@ org.apache.ignite.internal.processors.cache.CacheType
 org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQueryCloseableIterator
 org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQueryFutureIterator
 org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakReferenceCloseableIterator
-org.apache.ignite.internal.processors.cache.ChangeGlobalStateMessage
 org.apache.ignite.internal.processors.cache.ClientCacheChangeDiscoveryMessage
 org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessage
 org.apache.ignite.internal.processors.cache.ClusterCachesInfo$1$1
-org.apache.ignite.internal.processors.cache.ClusterState
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest
 org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy
@@ -880,10 +880,12 @@ org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Bool
 org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$DestroyBag
 org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree$Result
 org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO$EntryPart
+org.apache.ignite.internal.processors.cache.persistence.wal.AbstractWalRecordsIterator
 org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$FileArchiver$1
 org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager$RecordsIterator
 org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException
 org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException
+org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneWalRecordsIterator
 org.apache.ignite.internal.processors.cache.query.CacheQueryType
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryFuture$1
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$1
@@ -1032,9 +1034,13 @@ org.apache.ignite.internal.processors.closure.GridClosureProcessor$T8
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$T9
 org.apache.ignite.internal.processors.closure.GridClosureProcessor$TaskNoReduceAdapter
 org.apache.ignite.internal.processors.closure.GridPeerDeployAwareTaskAdapter
+org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage
+org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage
 org.apache.ignite.internal.processors.cluster.ClusterProcessor$3
 org.apache.ignite.internal.processors.cluster.ClusterProcessor$3$1
+org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState
 org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$1$1
+org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$2
 org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$3
 org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$4
 org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor$6
@@ -1876,6 +1882,7 @@ org.apache.ignite.internal.visor.node.VisorNodeSuppressedErrorsTask
 org.apache.ignite.internal.visor.node.VisorNodeSuppressedErrorsTask$VisorNodeSuppressedErrorsJob
 org.apache.ignite.internal.visor.node.VisorNodeSuppressedErrorsTaskArg
 org.apache.ignite.internal.visor.node.VisorPeerToPeerConfiguration
+org.apache.ignite.internal.visor.node.VisorPersistenceMetrics
 org.apache.ignite.internal.visor.node.VisorPersistentStoreConfiguration
 org.apache.ignite.internal.visor.node.VisorRestConfiguration
 org.apache.ignite.internal.visor.node.VisorSegmentationConfiguration


[19/41] ignite git commit: Fixed cache plugin validation.

Posted by sb...@apache.org.
Fixed cache plugin validation.


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

Branch: refs/heads/ignite-5578-1
Commit: 3314a4513c816c027dad80258ded1fe5a4c2b700
Parents: 3c88737
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 17:28:47 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 17:28:47 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/ClusterCachesInfo.java     | 30 +++++++++++++++++++-
 .../cache/DynamicCacheDescriptor.java           | 13 ---------
 2 files changed, 29 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3314a451/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
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 738e4ac..949bc19 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
@@ -37,6 +37,7 @@ import org.apache.ignite.cache.CacheExistsException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.GridCachePluginContext;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -53,6 +54,9 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 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.plugin.CachePluginContext;
+import org.apache.ignite.plugin.CachePluginProvider;
+import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.jetbrains.annotations.Nullable;
 
@@ -175,8 +179,32 @@ class ClusterCachesInfo {
                             locCacheInfo.cacheData().config().getName());
                     }
 
-                    if (checkConsistency)
+                    if (checkConsistency) {
                         checkCache(locCacheInfo, cacheData, cacheData.receivedFrom());
+
+                        ClusterNode rmt = ctx.discovery().node(cacheData.receivedFrom());
+
+                        if (rmt == null) {
+                            for (ClusterNode node : ctx.discovery().localJoin().discoCache().serverNodes()) {
+                                if (!node.isLocal() && ctx.discovery().cacheAffinityNode(node, locCfg.getName())) {
+                                    rmt = node;
+
+                                    break;
+                                }
+                            }
+                        }
+
+                        if (rmt != null) {
+                            for (PluginProvider p : ctx.plugins().allProviders()) {
+                                CachePluginContext pluginCtx = new GridCachePluginContext(ctx, locCfg);
+
+                                CachePluginProvider provider = p.createCacheProvider(pluginCtx);
+
+                                if (provider != null)
+                                    provider.validateRemote(locCfg, cacheData.cacheConfiguration(), rmt);
+                            }
+                        }
+                    }
                 }
 
                 if (checkConsistency)

http://git-wip-us.apache.org/repos/asf/ignite/blob/3314a451/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 315013d..18abcd8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -24,7 +24,6 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.QuerySchema;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -57,9 +56,6 @@ public class DynamicCacheDescriptor {
     /** Template configuration flag. */
     private boolean template;
 
-    /** Cache plugin manager. */
-    private final CachePluginManager pluginMgr;
-
     /** */
     private boolean updatesAllowed = true;
 
@@ -138,8 +134,6 @@ public class DynamicCacheDescriptor {
         this.sql = sql;
         this.deploymentId = deploymentId;
 
-        pluginMgr = new CachePluginManager(ctx, cacheCfg);
-
         cacheId = CU.cacheId(cacheCfg.getName());
 
         synchronized (schemaMux) {
@@ -242,13 +236,6 @@ public class DynamicCacheDescriptor {
     }
 
     /**
-     * @return Cache plugin manager.
-     */
-    public CachePluginManager pluginManager() {
-        return pluginMgr;
-    }
-
-    /**
      * @return Updates allowed flag.
      */
     public boolean updatesAllowed() {


[16/41] ignite git commit: Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2

Posted by sb...@apache.org.
Merge remote-tracking branch 'community/ignite-2.1.2' into ignite-2.1.2


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

Branch: refs/heads/ignite-5578-1
Commit: 55a5ca019db7d06aab2fb567dbc0da1f133d0fb7
Parents: a1cac1b 0d6fb1a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 15:55:38 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 15:55:38 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/database/InlineIndexHelper.java | 2 +-
 .../processors/query/h2/database/InlineIndexHelperTest.java      | 4 ++++
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[09/41] ignite git commit: IGNITE-5159: DDL example. This closes #2227.

Posted by sb...@apache.org.
IGNITE-5159: DDL example. This closes #2227.


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

Branch: refs/heads/ignite-5578-1
Commit: 2a5390b1c083819f059e449b34a5979ec35b7755
Parents: 99fd75d
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Jul 7 15:01:43 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 7 15:01:43 2017 +0300

----------------------------------------------------------------------
 .../examples/datagrid/CacheQueryDdlExample.java | 119 +++++++++++++++++++
 .../ignite/examples/CacheExamplesSelfTest.java  |   8 ++
 2 files changed, 127 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2a5390b1/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDdlExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDdlExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDdlExample.java
new file mode 100644
index 0000000..84a67cd
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryDdlExample.java
@@ -0,0 +1,119 @@
+/*
+ * 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.examples.datagrid;
+
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.ExampleNodeStartup;
+
+/**
+ * Example to showcase DDL capabilities of Ignite's SQL engine.
+ * <p>
+ * Remote nodes could be started from command line as follows:
+ * {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
+ * <p>
+ * Alternatively you can run {@link ExampleNodeStartup} in either same or another JVM.
+ */
+public class CacheQueryDdlExample {
+    /** Dummy cache name. */
+    private static final String DUMMY_CACHE_NAME = "dummy_cache";
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     * @throws Exception If example execution failed.
+     */
+    @SuppressWarnings({"unused", "ThrowFromFinallyBlock"})
+    public static void main(String[] args) throws Exception {
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            print("Cache query DDL example started.");
+
+            // Create dummy cache to act as an entry point for SQL queries (new SQL API which do not require this
+            // will appear in future versions, JDBC and ODBC drivers do not require it already).
+            CacheConfiguration<?, ?> cacheCfg = new CacheConfiguration<>(DUMMY_CACHE_NAME)
+                .setSqlSchema("PUBLIC").setIndexedTypes(Integer.class, Integer.class);
+
+            try (
+                IgniteCache<?, ?> cache = ignite.getOrCreateCache(cacheCfg)
+            ) {
+                // Create table based on PARTITIONED template with one backup.
+                cache.query(new SqlFieldsQuery(
+                    "CREATE TABLE person (id LONG PRIMARY KEY, name VARCHAR, city_id LONG) " +
+                    "WITH \"backups=1\"")).getAll();
+
+                // Create reference City table based on REPLICATED template.
+                cache.query(new SqlFieldsQuery(
+                    "CREATE TABLE city (id LONG PRIMARY KEY, name VARCHAR) WITH \"template=replicated\"")).getAll();
+
+                // Create an index.
+                cache.query(new SqlFieldsQuery("CREATE INDEX on Person (city_id)")).getAll();
+
+                print("Created database objects.");
+
+                SqlFieldsQuery qry = new SqlFieldsQuery("INSERT INTO person (id, name, city_id) values (?, ?, ?)");
+
+                cache.query(qry.setArgs(1L, "John Doe", 3L)).getAll();
+                cache.query(qry.setArgs(2L, "Jane Roe", 2L)).getAll();
+                cache.query(qry.setArgs(3L, "Mary Major", 1L)).getAll();
+                cache.query(qry.setArgs(4L, "Richard Miles", 2L)).getAll();
+
+                qry = new SqlFieldsQuery("INSERT INTO city (id, name) VALUES (?, ?)");
+
+                cache.query(qry.setArgs(1L, "Forest Hill")).getAll();
+                cache.query(qry.setArgs(2L, "Denver")).getAll();
+                cache.query(qry.setArgs(3L, "St. Petersburg")).getAll();
+
+                print("Populated data.");
+
+                List<List<?>> res = cache.query(new SqlFieldsQuery(
+                    "SELECT p.name, c.name FROM Person p INNER JOIN City c on c.id = p.city_id")).getAll();
+
+                print("Query results:");
+
+                for (Object next : res)
+                    System.out.println(">>>     " + next);
+
+                cache.query(new SqlFieldsQuery("drop table Person")).getAll();
+                cache.query(new SqlFieldsQuery("drop table City")).getAll();
+
+                print("Dropped database objects.");
+            }
+            finally {
+                // Distributed cache can be removed from cluster only by #destroyCache() call.
+                ignite.destroyCache(DUMMY_CACHE_NAME);
+            }
+
+            print("Cache query DDL example finished.");
+        }
+    }
+
+    /**
+     * Prints message.
+     *
+     * @param msg Message to print before all objects are printed.
+     */
+    private static void print(String msg) {
+        System.out.println();
+        System.out.println(">>> " + msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2a5390b1/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java b/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
index 3447dff..f65d97c 100644
--- a/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
+++ b/examples/src/test/java/org/apache/ignite/examples/CacheExamplesSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.examples.datagrid.CacheApiExample;
 import org.apache.ignite.examples.datagrid.CacheContinuousQueryExample;
 import org.apache.ignite.examples.datagrid.CacheDataStreamerExample;
 import org.apache.ignite.examples.datagrid.CachePutGetExample;
+import org.apache.ignite.examples.datagrid.CacheQueryDdlExample;
 import org.apache.ignite.examples.datagrid.CacheQueryDmlExample;
 import org.apache.ignite.examples.datagrid.CacheQueryExample;
 import org.apache.ignite.examples.datagrid.CacheTransactionExample;
@@ -138,6 +139,13 @@ public class CacheExamplesSelfTest extends GridAbstractExamplesTest {
     /**
      * @throws Exception If failed.
      */
+    public void testCacheQUeryDdlExample() throws Exception {
+        CacheQueryDdlExample.main(EMPTY_ARGS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testCacheApiExample() throws Exception {
         CacheApiExample.main(EMPTY_ARGS);
     }


[34/41] ignite git commit: IGNITE-5716 .NET: Fix 2-byte field offset handling - improve tests

Posted by sb...@apache.org.
IGNITE-5716 .NET: Fix 2-byte field offset handling - improve tests


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

Branch: refs/heads/ignite-5578-1
Commit: f1c8e59cb9410915a6e61ba6f4f63c6f3c795c75
Parents: 313f86e
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jul 10 13:15:20 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jul 10 13:15:20 2017 +0300

----------------------------------------------------------------------
 .../Binary/BinaryFooterTest.cs                  | 32 ++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f1c8e59c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
index 36f2f65..5088e5a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryFooterTest.cs
@@ -20,6 +20,8 @@ namespace Apache.Ignite.Core.Tests.Binary
     using System;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using NUnit.Framework;
@@ -117,16 +119,46 @@ namespace Apache.Ignite.Core.Tests.Binary
                         Assert.AreEqual(dt.Arr, r.Arr);
                         Assert.AreEqual(dt.Int, r.Int);
                     }
+
+                    TestSql(dt, getMarsh());
                 }
             }
         }
 
         /// <summary>
+        /// Tests SQL query, which verifies Java side of things.
+        /// </summary>
+        private static void TestSql(OffsetTest dt, Marshaller marsh)
+        {
+            var ignite = marsh.Ignite;
+
+            if (ignite == null)
+            {
+                return;
+            }
+
+            var cache = ignite.GetOrCreateCache<int, OffsetTest>(
+                    new CacheConfiguration("offs", new QueryEntity(typeof(int), typeof(OffsetTest))));
+
+            // Cache operation.
+            cache[1] = dt;
+            Assert.AreEqual(dt.Int, cache[1].Int);
+            Assert.AreEqual(dt.Arr, cache[1].Arr);
+
+            // SQL: read field on Java side to ensure correct offset handling.
+            var res = cache.QueryFields(new SqlFieldsQuery("select int from OffsetTest")).GetAll()[0][0];
+            Assert.AreEqual(dt.Int, (int) res);
+        }
+
+        /// <summary>
         /// Offset test.
         /// </summary>
         private class OffsetTest : IBinarizable
         {
+            [QuerySqlField]
             public byte[] Arr;  // Array to enforce field offset.
+
+            [QuerySqlField]
             public int Int;     // Value at offset.
 
             public void WriteBinary(IBinaryWriter writer)


[12/41] ignite git commit: 2.1 Fixed assert (cacheName can be null here).

Posted by sb...@apache.org.
2.1 Fixed assert (cacheName can be null here).


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

Branch: refs/heads/ignite-5578-1
Commit: 13399d530f3458c3bf59634857cb05f801e8cc34
Parents: d232648
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 7 15:24:09 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 7 15:24:09 2017 +0300

----------------------------------------------------------------------
 .../processors/datastructures/DataStructuresProcessor.java     | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/13399d53/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 4399fe2..5564b79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -326,12 +326,10 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
      * @return {@code True} if cache with such name is used to store data structures.
      */
     public static boolean isDataStructureCache(String cacheName) {
-        assert cacheName != null;
-
-        return cacheName.startsWith(ATOMICS_CACHE_NAME) ||
+        return cacheName != null && (cacheName.startsWith(ATOMICS_CACHE_NAME) ||
             cacheName.startsWith(DS_CACHE_NAME_PREFIX) ||
             cacheName.equals(DEFAULT_DS_GROUP_NAME) ||
-            cacheName.equals(DEFAULT_VOLATILE_DS_GROUP_NAME);
+            cacheName.equals(DEFAULT_VOLATILE_DS_GROUP_NAME));
     }
 
     /**


[32/41] ignite git commit: IGNITE-5491 .NET: PersistentStoreMetrics

Posted by sb...@apache.org.
IGNITE-5491 .NET: PersistentStoreMetrics


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

Branch: refs/heads/ignite-5578-1
Commit: b085fa0db67f8cda01e6927715241b10ac43c0f2
Parents: 9790a46
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jul 10 12:24:54 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jul 10 12:24:54 2017 +0300

----------------------------------------------------------------------
 .../platform/cluster/PlatformClusterGroup.java  |  39 ++++-
 .../utils/PlatformConfigurationUtils.java       |   8 +-
 .../Cache/PersistentStoreTest.cs                |   9 +-
 .../IgniteConfigurationSerializerTest.cs        |  11 +-
 .../IgniteConfigurationTest.cs                  |  11 +-
 .../Apache.Ignite.Core.csproj                   |   2 +
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   9 ++
 .../IgniteConfigurationSection.xsd              |  15 ++
 .../Impl/Cluster/ClusterGroupImpl.cs            |  14 ++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   7 +
 .../PersistentStore/PersistentStoreMetrics.cs   |  88 +++++++++++
 .../PersistentStore/IPersistentStoreMetrics.cs  |  87 +++++++++++
 .../PersistentStoreConfiguration.cs             | 156 ++++++++++++-------
 13 files changed, 392 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index 3e3aa3a..f6e3d2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -25,6 +25,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCluster;
 import org.apache.ignite.MemoryMetrics;
+import org.apache.ignite.PersistenceMetrics;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
@@ -122,6 +123,9 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
     private static final int OP_IS_ACTIVE = 29;
 
+    /** */
+    private static final int OP_PERSISTENT_STORE_METRICS = 30;
+
     /** Projection. */
     private final ClusterGroupEx prj;
 
@@ -146,7 +150,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
                 break;
 
-            case OP_MEMORY_METRICS:
+            case OP_MEMORY_METRICS: {
                 Collection<MemoryMetrics> metrics = prj.ignite().memoryMetrics();
 
                 writer.writeInt(metrics.size());
@@ -156,6 +160,15 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 }
 
                 break;
+            }
+
+            case OP_PERSISTENT_STORE_METRICS: {
+                PersistenceMetrics metrics = prj.ignite().persistentStoreMetrics();
+
+                writePersistentStoreMetrics(writer, metrics);
+
+                break;
+            }
 
             default:
                 super.processOutStream(type, writer);
@@ -448,4 +461,28 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
         writer.writeFloat(metrics.getLargeEntriesPagesPercentage());
         writer.writeFloat(metrics.getPagesFillFactor());
     }
+
+    /**
+     * Writes persistent store metrics.
+     *
+     * @param writer Writer.
+     * @param metrics Metrics
+     */
+    private void writePersistentStoreMetrics(BinaryRawWriter writer, PersistenceMetrics metrics) {
+        assert writer != null;
+        assert metrics != null;
+
+        writer.writeFloat(metrics.getWalLoggingRate());
+        writer.writeFloat(metrics.getWalWritingRate());
+        writer.writeInt(metrics.getWalArchiveSegments());
+        writer.writeFloat(metrics.getWalFsyncTimeAverage());
+        writer.writeLong(metrics.getLastCheckpointingDuration());
+        writer.writeLong(metrics.getLastCheckpointLockWaitDuration());
+        writer.writeLong(metrics.getLastCheckpointMarkDuration());
+        writer.writeLong(metrics.getLastCheckpointPagesWriteDuration());
+        writer.writeLong(metrics.getLastCheckpointFsyncDuration());
+        writer.writeLong(metrics.getLastCheckpointTotalPagesNumber());
+        writer.writeLong(metrics.getLastCheckpointDataPagesNumber());
+        writer.writeLong(metrics.getLastCheckpointCopiedOnWritePagesNumber());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
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 92db41a..d513071 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
@@ -1505,7 +1505,10 @@ public class PlatformConfigurationUtils {
                 .setWalFlushFrequency((int) in.readLong())
                 .setWalFsyncDelay(in.readInt())
                 .setWalRecordIteratorBufferSize(in.readInt())
-                .setAlwaysWriteFullPages(in.readBoolean());
+                .setAlwaysWriteFullPages(in.readBoolean())
+                .setMetricsEnabled(in.readBoolean())
+                .setSubIntervals(in.readInt())
+                .setRateTimeInterval(in.readLong());
     }
 
     /**
@@ -1535,6 +1538,9 @@ public class PlatformConfigurationUtils {
             w.writeInt(cfg.getWalFsyncDelay());
             w.writeInt(cfg.getWalRecordIteratorBufferSize());
             w.writeBoolean(cfg.isAlwaysWriteFullPages());
+            w.writeBoolean(cfg.isMetricsEnabled());
+            w.writeInt(cfg.getSubIntervals());
+            w.writeLong(cfg.getRateTimeInterval());
 
         } else {
             w.writeBoolean(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
index e9cbce8..3320dd7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
@@ -58,7 +58,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                 {
                     PersistentStorePath = Path.Combine(_tempDir, "Store"),
                     WalStorePath = Path.Combine(_tempDir, "WalStore"),
-                    WalArchivePath = Path.Combine(_tempDir, "WalArchive")
+                    WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
+                    MetricsEnabled = true
                 }
             };
 
@@ -72,6 +73,12 @@ namespace Apache.Ignite.Core.Tests.Cache
                 var cache = ignite.CreateCache<int, int>(cacheName);
 
                 cache[1] = 1;
+
+                // Check some metrics.
+                var metrics = ignite.GetPersistentStoreMetrics();
+                Assert.Greater(metrics.WalLoggingRate, 0);
+                Assert.Greater(metrics.WalWritingRate, 0);
+                Assert.Greater(metrics.WalFsyncTimeAverage, 0);
             }
 
             // Verify directories.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 3d0bb56..7e0d941 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -143,7 +143,7 @@ namespace Apache.Ignite.Core.Tests
                                 </memoryPolicies>
                             </memoryConfiguration>
                             <sqlConnectorConfiguration host='bar' port='10' portRange='11' socketSendBufferSize='12' socketReceiveBufferSize='13' tcpNoDelay='true' maxOpenCursorsPerConnection='14' threadPoolSize='15' />
-                            <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' walStorePath='baz' />
+                            <persistentStoreConfiguration alwaysWriteFullPages='true' checkpointingFrequency='00:00:1' checkpointingPageBufferSize='2' checkpointingThreads='3' lockWaitTime='00:00:04' persistentStorePath='foo' tlbSize='5' walArchivePath='bar' walFlushFrequency='00:00:06' walFsyncDelayNanos='7' walHistorySize='8' walMode='None' walRecordIteratorBufferSize='9' walSegments='10' walSegmentSize='11' walStorePath='baz' metricsEnabled='true' rateTimeInterval='0:0:6' subIntervals='3' />
                         </igniteConfig>";
 
             var cfg = IgniteConfiguration.FromXml(xml);
@@ -315,7 +315,9 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(10, pers.WalSegments);
             Assert.AreEqual(11, pers.WalSegmentSize);
             Assert.AreEqual("baz", pers.WalStorePath);
-
+            Assert.IsTrue(pers.MetricsEnabled);
+            Assert.AreEqual(3, pers.SubIntervals);
+            Assert.AreEqual(TimeSpan.FromSeconds(6), pers.RateTimeInterval);
         }
 
         /// <summary>
@@ -907,7 +909,10 @@ namespace Apache.Ignite.Core.Tests
                     WalRecordIteratorBufferSize = 32 * 1024 * 1024,
                     WalSegments = 6,
                     WalSegmentSize = 5 * 1024 * 1024,
-                    WalStorePath = Path.GetTempPath()
+                    WalStorePath = Path.GetTempPath(),
+                    SubIntervals = 25,
+                    MetricsEnabled = true,
+                    RateTimeInterval = TimeSpan.FromDays(1)
                 },
                 IsActiveOnStart = false
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 5e5cb1c..4902118 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -266,6 +266,9 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(pers.WalSegments, resPers.WalSegments);
                 Assert.AreEqual(pers.WalSegmentSize, resPers.WalSegmentSize);
                 Assert.AreEqual(pers.WalStorePath, resPers.WalStorePath);
+                Assert.AreEqual(pers.MetricsEnabled, resPers.MetricsEnabled);
+                Assert.AreEqual(pers.RateTimeInterval, resPers.RateTimeInterval);
+                Assert.AreEqual(pers.SubIntervals, resPers.SubIntervals);
             }
         }
 
@@ -539,6 +542,9 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(PersistentStoreConfiguration.DefaultWalSegmentSize, cfg.WalSegmentSize);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultWalSegments, cfg.WalSegments);
             Assert.AreEqual(WalMode.Default, cfg.WalMode);
+            Assert.IsFalse(cfg.MetricsEnabled);
+            Assert.AreEqual(PersistentStoreConfiguration.DefaultSubIntervals, cfg.SubIntervals);
+            Assert.AreEqual(PersistentStoreConfiguration.DefaultRateTimeInterval, cfg.RateTimeInterval);
         }
 
         /// <summary>
@@ -741,7 +747,10 @@ namespace Apache.Ignite.Core.Tests
                     WalRecordIteratorBufferSize = 32 * 1024 * 1024,
                     WalSegments = 6,
                     WalSegmentSize = 5 * 1024 * 1024,
-                    WalStorePath = Path.GetTempPath()
+                    WalStorePath = Path.GetTempPath(),
+                    MetricsEnabled = true,
+                    SubIntervals = 7,
+                    RateTimeInterval = TimeSpan.FromSeconds(9)
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 9df2889..76132c3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -99,6 +99,8 @@
     <Compile Include="Cache\IMemoryMetrics.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
+    <Compile Include="Impl\PersistentStore\PersistentStoreMetrics.cs" />
+    <Compile Include="PersistentStore\IPersistentStoreMetrics.cs" />
     <Compile Include="PersistentStore\Package-Info.cs" />
     <Compile Include="PersistentStore\PersistentStoreConfiguration.cs" />
     <Compile Include="Configuration\SqlConnectorConfiguration.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
index 8c4bee2..bf061db 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -32,6 +32,7 @@ namespace Apache.Ignite.Core
     using Apache.Ignite.Core.Log;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.PersistentStore;
     using Apache.Ignite.Core.Plugin;
     using Apache.Ignite.Core.Services;
     using Apache.Ignite.Core.Transactions;
@@ -359,5 +360,13 @@ namespace Apache.Ignite.Core
         ///   <c>true</c> if the grid is active; otherwise, <c>false</c>.
         /// </returns>
         bool IsActive();
+
+        /// <summary>
+        /// Gets the persistent store metrics.
+        /// <para />
+        /// To enable metrics set <see cref="PersistentStoreConfiguration.MetricsEnabled"/> property
+        /// in <see cref="IgniteConfiguration.PersistentStoreConfiguration"/>.
+        /// </summary>
+        IPersistentStoreMetrics GetPersistentStoreMetrics();
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 919f303..d7fd5ac 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1394,6 +1394,21 @@
                                 <xs:documentation>Whether full pages should always be written.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
+                        <xs:attribute name="metricsEnabled" type="xs:boolean">
+                            <xs:annotation>
+                                <xs:documentation>Enable persistent store metrics.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="subIntervals" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Number of sub intervals to split RateTimeInterval into.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="rateTimeInterval" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Rate time interval.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
                     </xs:complexType>
                 </xs:element>
                 <xs:element name="pluginConfigurations" minOccurs="0">

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index 37b4e79..6e07b78 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -35,9 +35,11 @@ namespace Apache.Ignite.Core.Impl.Cluster
     using Apache.Ignite.Core.Impl.Compute;
     using Apache.Ignite.Core.Impl.Events;
     using Apache.Ignite.Core.Impl.Messaging;
+    using Apache.Ignite.Core.Impl.PersistentStore;
     using Apache.Ignite.Core.Impl.Services;
     using Apache.Ignite.Core.Impl.Unmanaged;
     using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.PersistentStore;
     using Apache.Ignite.Core.Services;
     using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
 
@@ -130,6 +132,9 @@ namespace Apache.Ignite.Core.Impl.Cluster
         /** */
         private const int OpIsActive = 29;
 
+        /** */
+        private const int OpGetPersistentStoreMetrics = 30;
+
         /** Initial Ignite instance. */
         private readonly Ignite _ignite;
         
@@ -615,6 +620,15 @@ namespace Apache.Ignite.Core.Impl.Cluster
         }
 
         /// <summary>
+        /// Gets the persistent store metrics.
+        /// </summary>
+        public IPersistentStoreMetrics GetPersistentStoreMetrics()
+        {
+            return DoInOp(OpGetPersistentStoreMetrics, stream =>
+                new PersistentStoreMetrics(Marshaller.StartUnmarshal(stream, false)));
+        }
+
+        /// <summary>
         /// Creates new Cluster Group from given native projection.
         /// </summary>
         /// <param name="prj">Native projection.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index fc7894a..205f6e2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -45,6 +45,7 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
     using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.PersistentStore;
     using Apache.Ignite.Core.Services;
     using Apache.Ignite.Core.Transactions;
     using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
@@ -757,6 +758,12 @@ namespace Apache.Ignite.Core.Impl
             return _prj.IsActive();
         }
 
+        /** <inheritdoc /> */
+        public IPersistentStoreMetrics GetPersistentStoreMetrics()
+        {
+            return _prj.GetPersistentStoreMetrics();
+        }
+
         /// <summary>
         /// Gets or creates near cache.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
new file mode 100644
index 0000000..85a4fdf
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PersistentStore/PersistentStoreMetrics.cs
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.PersistentStore
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.PersistentStore;
+    
+    /// <summary>
+    /// Persistent store metrics.
+    /// </summary>
+    internal class PersistentStoreMetrics : IPersistentStoreMetrics
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="PersistentStoreMetrics"/> class.
+        /// </summary>
+        public PersistentStoreMetrics(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            WalLoggingRate = reader.ReadFloat();
+            WalWritingRate = reader.ReadFloat();
+            WalArchiveSegments = reader.ReadInt();
+            WalFsyncTimeAverage = reader.ReadFloat();
+            LastCheckpointingDuration = reader.ReadLongAsTimespan();
+            LastCheckpointLockWaitDuration = reader.ReadLongAsTimespan();
+            LastCheckpointMarkDuration = reader.ReadLongAsTimespan();
+            LastCheckpointPagesWriteDuration = reader.ReadLongAsTimespan();
+            LastCheckpointFsyncDuration = reader.ReadLongAsTimespan();
+            LastCheckpointTotalPagesNumber = reader.ReadLong();
+            LastCheckpointDataPagesNumber = reader.ReadLong();
+            LastCheckpointCopiedOnWritePagesNumber = reader.ReadLong();
+        }
+
+        /** <inheritdoc /> */
+        public float WalLoggingRate { get; private set; }
+
+        /** <inheritdoc /> */
+        public float WalWritingRate { get; private set; }
+
+        /** <inheritdoc /> */
+        public int WalArchiveSegments { get; private set; }
+
+        /** <inheritdoc /> */
+        public float WalFsyncTimeAverage { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointingDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointLockWaitDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointMarkDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointPagesWriteDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public TimeSpan LastCheckpointFsyncDuration { get; private set; }
+
+        /** <inheritdoc /> */
+        public long LastCheckpointTotalPagesNumber { get; private set; }
+
+        /** <inheritdoc /> */
+        public long LastCheckpointDataPagesNumber { get; private set; }
+
+        /** <inheritdoc /> */
+        public long LastCheckpointCopiedOnWritePagesNumber { get; private set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
new file mode 100644
index 0000000..e7e8481
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/IPersistentStoreMetrics.cs
@@ -0,0 +1,87 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.PersistentStore
+{
+    using System;
+
+    /// <summary>
+    /// Persistent store metrics.
+    /// </summary>
+    public interface IPersistentStoreMetrics
+    {
+        /// <summary>
+        /// Gets the average number of WAL records per second written during the last time interval. 
+        /// </summary>
+        float WalLoggingRate { get; }
+
+        /// <summary>
+        /// Gets the average number of bytes per second written during the last time interval.
+        /// </summary>
+        float WalWritingRate { get; }
+
+        /// <summary>
+        /// Gets the current number of WAL segments in the WAL archive.
+        /// </summary>
+        int WalArchiveSegments { get; }
+
+        /// <summary>
+        /// Gets the average WAL fsync duration in microseconds over the last time interval.
+        /// </summary>
+        float WalFsyncTimeAverage { get; }
+
+        /// <summary>
+        /// Gets the duration of the last checkpoint.
+        /// </summary>
+        TimeSpan LastCheckpointingDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of last checkpoint lock wait.
+        /// </summary>
+        TimeSpan LastCheckpointLockWaitDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of last checkpoint mark phase.
+        /// </summary>
+        TimeSpan LastCheckpointMarkDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of last checkpoint pages write phase.
+        /// </summary>
+        TimeSpan LastCheckpointPagesWriteDuration { get; }
+
+        /// <summary>
+        /// Gets the duration of the sync phase of the last checkpoint.
+        /// </summary>
+        TimeSpan LastCheckpointFsyncDuration { get; }
+
+        /// <summary>
+        /// Gets the total number of pages written during the last checkpoint.
+        /// </summary>
+        long LastCheckpointTotalPagesNumber { get; }
+
+        /// <summary>
+        /// Gets the number of data pages written during the last checkpoint.
+        /// </summary>
+        long LastCheckpointDataPagesNumber { get; }
+
+        /// <summary>
+        /// Gets the number of pages copied to a temporary checkpoint buffer during the last checkpoint.
+        /// </summary>
+        long LastCheckpointCopiedOnWritePagesNumber { get; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b085fa0d/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
index c998ab3..43b17ac 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.PersistentStore
     using System;
     using System.ComponentModel;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary;
 
@@ -29,6 +30,73 @@ namespace Apache.Ignite.Core.PersistentStore
     public class PersistentStoreConfiguration
     {
         /// <summary>
+        /// Default value for <see cref="CheckpointingPageBufferSize"/>.
+        /// </summary>
+        public const long DefaultCheckpointingPageBufferSize = 256L * 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="CheckpointingThreads"/>.
+        /// </summary>
+        public const int DefaultCheckpointingThreads = 1;
+
+        /// <summary>
+        /// Default value for <see cref="CheckpointingFrequency"/>.
+        /// </summary>
+        public static readonly TimeSpan DefaultCheckpointingFrequency = TimeSpan.FromSeconds(180);
+
+        /// <summary>
+        /// Default value for <see cref="LockWaitTime"/>.
+        /// </summary>
+        public static readonly TimeSpan DefaultLockWaitTime = TimeSpan.FromSeconds(10);
+
+        /// <summary>
+        /// Default value for <see cref="WalHistorySize"/>.
+        /// </summary>
+        public const int DefaultWalHistorySize = 20;
+
+        /// <summary>
+        /// Default value for <see cref="WalSegments"/>.
+        /// </summary>
+        public const int DefaultWalSegments = 10;
+
+        /// <summary>
+        /// Default value for <see cref="WalSegmentSize"/>.
+        /// </summary>
+        public const int DefaultWalSegmentSize = 64 * 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="TlbSize"/>.
+        /// </summary>
+        public const int DefaultTlbSize = 128 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="WalFlushFrequency"/>.
+        /// </summary>
+        public static readonly TimeSpan DefaultWalFlushFrequency = TimeSpan.FromSeconds(2);
+
+        /// <summary>
+        /// Default value for <see cref="WalRecordIteratorBufferSize"/>.
+        /// </summary>
+        public const int DefaultWalRecordIteratorBufferSize = 64 * 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="WalFsyncDelayNanos"/>.
+        /// </summary>
+        public const int DefaultWalFsyncDelayNanos = 1;
+
+        /// <summary>
+        /// The default sub intervals.
+        /// </summary>
+        [SuppressMessage("Microsoft.Naming", "CA1702:CompoundWordsShouldBeCasedCorrectly",
+            Justification = "Consistency with Java config")]
+        public const int DefaultSubIntervals = 5;
+
+        /// <summary>
+        /// The default rate time interval.
+        /// </summary>
+        public static readonly TimeSpan DefaultRateTimeInterval = TimeSpan.FromSeconds(60);
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="PersistentStoreConfiguration"/> class.
         /// </summary>
         public PersistentStoreConfiguration()
@@ -44,6 +112,8 @@ namespace Apache.Ignite.Core.PersistentStore
             WalFlushFrequency = DefaultWalFlushFrequency;
             WalRecordIteratorBufferSize = DefaultWalRecordIteratorBufferSize;
             WalFsyncDelayNanos = DefaultWalFsyncDelayNanos;
+            RateTimeInterval = DefaultRateTimeInterval;
+            SubIntervals = DefaultSubIntervals;
         }
 
         /// <summary>
@@ -64,12 +134,15 @@ namespace Apache.Ignite.Core.PersistentStore
             WalSegmentSize = reader.ReadInt();
             WalStorePath = reader.ReadString();
             WalArchivePath = reader.ReadString();
-            WalMode = (WalMode) reader.ReadInt();
+            WalMode = (WalMode)reader.ReadInt();
             TlbSize = reader.ReadInt();
             WalFlushFrequency = reader.ReadLongAsTimespan();
             WalFsyncDelayNanos = reader.ReadInt();
             WalRecordIteratorBufferSize = reader.ReadInt();
             AlwaysWriteFullPages = reader.ReadBoolean();
+            MetricsEnabled = reader.ReadBoolean();
+            SubIntervals = reader.ReadInt();
+            RateTimeInterval = reader.ReadLongAsTimespan();
         }
 
         /// <summary>
@@ -90,70 +163,18 @@ namespace Apache.Ignite.Core.PersistentStore
             writer.WriteInt(WalSegmentSize);
             writer.WriteString(WalStorePath);
             writer.WriteString(WalArchivePath);
-            writer.WriteInt((int) WalMode);
+            writer.WriteInt((int)WalMode);
             writer.WriteInt(TlbSize);
             writer.WriteTimeSpanAsLong(WalFlushFrequency);
             writer.WriteInt(WalFsyncDelayNanos);
             writer.WriteInt(WalRecordIteratorBufferSize);
             writer.WriteBoolean(AlwaysWriteFullPages);
+            writer.WriteBoolean(MetricsEnabled);
+            writer.WriteInt(SubIntervals);
+            writer.WriteTimeSpanAsLong(RateTimeInterval);
         }
 
         /// <summary>
-        /// Default value for <see cref="CheckpointingPageBufferSize"/>.
-        /// </summary>
-        public const long DefaultCheckpointingPageBufferSize = 256L * 1024 * 1024;
-
-        /// <summary>
-        /// Default value for <see cref="CheckpointingThreads"/>.
-        /// </summary>
-        public const int DefaultCheckpointingThreads = 1;
-
-        /// <summary>
-        /// Default value for <see cref="CheckpointingFrequency"/>.
-        /// </summary>
-        public static readonly TimeSpan DefaultCheckpointingFrequency = TimeSpan.FromSeconds(180);
-
-        /// <summary>
-        /// Default value for <see cref="LockWaitTime"/>.
-        /// </summary>
-        public static readonly TimeSpan DefaultLockWaitTime = TimeSpan.FromSeconds(10);
-
-        /// <summary>
-        /// Default value for <see cref="WalHistorySize"/>.
-        /// </summary>
-        public const int DefaultWalHistorySize = 20;
-
-        /// <summary>
-        /// Default value for <see cref="WalSegments"/>.
-        /// </summary>
-        public const int DefaultWalSegments = 10;
-
-        /// <summary>
-        /// Default value for <see cref="WalSegmentSize"/>.
-        /// </summary>
-        public const int DefaultWalSegmentSize = 64 * 1024 * 1024;
-
-        /// <summary>
-        /// Default value for <see cref="TlbSize"/>.
-        /// </summary>
-        public const int DefaultTlbSize = 128 * 1024;
-
-        /// <summary>
-        /// Default value for <see cref="WalFlushFrequency"/>.
-        /// </summary>
-        public static readonly TimeSpan DefaultWalFlushFrequency = TimeSpan.FromSeconds(2);
-
-        /// <summary>
-        /// Default value for <see cref="WalRecordIteratorBufferSize"/>.
-        /// </summary>
-        public const int DefaultWalRecordIteratorBufferSize = 64 * 1024 * 1024;
-
-        /// <summary>
-        /// Default value for <see cref="WalFsyncDelayNanos"/>.
-        /// </summary>
-        public const int DefaultWalFsyncDelayNanos = 1;
-
-        /// <summary>
         /// Gets or sets the path where data and indexes will be persisted.
         /// </summary>
         public string PersistentStorePath { get; set; }
@@ -247,5 +268,26 @@ namespace Apache.Ignite.Core.PersistentStore
         /// Gets or sets a value indicating whether full pages should always be written.
         /// </summary>
         public bool AlwaysWriteFullPages { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether to enable persistent store metrics.
+        /// See <see cref="IIgnite.GetPersistentStoreMetrics"/>.
+        /// </summary>
+        public bool MetricsEnabled { get; set; }
+
+        /// <summary>
+        /// Gets or sets the length of the time interval for rate-based metrics.
+        /// This interval defines a window over which hits will be tracked.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "00:01:00")]
+        public TimeSpan RateTimeInterval { get; set; }
+
+        /// <summary>
+        /// Number of sub-intervals to split the <see cref="RateTimeInterval"/> into to track the update history.
+        /// </summary>
+        [DefaultValue(DefaultSubIntervals)]
+        [SuppressMessage("Microsoft.Naming", "CA1702:CompoundWordsShouldBeCasedCorrectly",
+            Justification = "Consistency with Java config")]
+        public int SubIntervals { get; set; }
     }
 }


[02/41] ignite git commit: IGNITE-5520 - Fixed IgniteChangeGlobalStateFailoverTest

Posted by sb...@apache.org.
IGNITE-5520 - Fixed IgniteChangeGlobalStateFailoverTest


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

Branch: refs/heads/ignite-5578-1
Commit: 85d8c6572d78c7d6796906d92b566e2fa5b116be
Parents: ab52671
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Fri Jul 7 13:13:55 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Jul 7 13:15:26 2017 +0300

----------------------------------------------------------------------
 .../IgniteChangeGlobalStateFailOverTest.java    | 56 +++++++++++++-------
 1 file changed, 37 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/85d8c657/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
index 92d1f21..02a21f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateFailOverTest.java
@@ -21,11 +21,11 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static java.lang.Thread.sleep;
 import static org.apache.ignite.testframework.GridTestUtils.runAsync;
@@ -144,8 +144,6 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
      * @throws Exception If failed.
      */
     public void testActivateDeActivateOnJoiningNode() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5520");
-
         final Ignite igB1 = backUp(0);
         final Ignite igB2 = backUp(1);
         final Ignite igB3 = backUp(2);
@@ -162,14 +160,17 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        final AtomicBoolean canAct = new AtomicBoolean(true);
         final AtomicInteger seqIdx = new AtomicInteger(backUpNodes());
 
+        final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
+
         try {
             final IgniteInternalFuture<Void> af = runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     while (!stop.get()) {
-                        if (canAct.get()) {
+                        rwLock.readLock().lock();
+
+                        try {
                             Ignite ig = randomBackUp(false);
 
                             long start = System.currentTimeMillis();
@@ -182,13 +183,12 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
 
                             for (Ignite ign : allBackUpNodes())
                                 assertTrue(ign.active());
-
-                            canAct.set(false);
                         }
-                        else
-                            U.sleep(100);
-
+                        finally {
+                            rwLock.readLock().unlock();
+                        }
                     }
+
                     return null;
                 }
             });
@@ -196,7 +196,9 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
             final IgniteInternalFuture<Void> df = runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     while (!stop.get()) {
-                        if (!canAct.get()) {
+                        rwLock.writeLock().lock();
+
+                        try {
                             Ignite ig = randomBackUp(false);
 
                             long start = System.currentTimeMillis();
@@ -209,20 +211,28 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
 
                             for (Ignite ign : allBackUpNodes())
                                 assertTrue(!ign.active());
-
-                            canAct.set(true);
                         }
-                        else
-                            U.sleep(100);
+                        finally {
+                            rwLock.writeLock().unlock();
+                        }
                     }
+
                     return null;
                 }
             });
 
             IgniteInternalFuture<Void> jf1 = runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
-                    while (!stop.get())
-                        startBackUp(seqIdx.incrementAndGet());
+                    while (!stop.get()) {
+                        rwLock.readLock().lock();
+
+                        try {
+                            startBackUp(seqIdx.incrementAndGet());
+                        }
+                        finally {
+                            rwLock.readLock().unlock();
+                        }
+                    }
 
                     return null;
                 }
@@ -230,8 +240,16 @@ public class IgniteChangeGlobalStateFailOverTest extends IgniteChangeGlobalState
 
             IgniteInternalFuture<Void> jf2 = runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
-                    while (!stop.get())
-                        startBackUp(seqIdx.incrementAndGet());
+                    while (!stop.get()) {
+                        rwLock.readLock().lock();
+
+                        try {
+                            startBackUp(seqIdx.incrementAndGet());
+                        }
+                        finally {
+                            rwLock.readLock().unlock();
+                        }
+                    }
 
                     return null;
                 }


[27/41] ignite git commit: GG-12466 - Clean up public API for snapshots

Posted by sb...@apache.org.
GG-12466 - Clean up public API for snapshots


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

Branch: refs/heads/ignite-5578-1
Commit: 36716fb1f4de5d1f5baab1e27afc95583dd4c7b4
Parents: 17904cb
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Mon Jul 10 11:09:43 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 10 11:10:41 2017 +0300

----------------------------------------------------------------------
 ...ishSnapshotOperationAckDiscoveryMessage.java |  84 -------
 .../snapshot/SnapshotCheckParameters.java       |  75 ------
 .../pagemem/snapshot/SnapshotOperation.java     | 235 -------------------
 .../pagemem/snapshot/SnapshotOperationType.java |  49 ----
 ...artSnapshotOperationAckDiscoveryMessage.java | 149 ------------
 .../StartSnapshotOperationDiscoveryMessage.java | 192 ---------------
 .../GridCachePartitionExchangeManager.java      |   9 +-
 .../processors/cache/GridCacheProcessor.java    |  22 +-
 .../cache/GridCacheSharedContext.java           |   6 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   8 +-
 .../GridDhtPartitionsExchangeFuture.java        |  54 ++---
 .../GridCacheDatabaseSharedManager.java         |   3 +-
 .../persistence/IgniteCacheSnapshotManager.java | 150 ------------
 .../persistence/file/FilePageStoreManager.java  |   2 +-
 .../snapshot/IgniteCacheSnapshotManager.java    | 161 +++++++++++++
 .../snapshot/SnapshotDiscoveryMessage.java      |  33 +++
 .../persistence/snapshot/SnapshotOperation.java |  44 ++++
 .../resources/META-INF/classnames.properties    |   5 -
 .../loadtests/hashmap/GridCacheTestContext.java |   6 +-
 19 files changed, 281 insertions(+), 1006 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/FinishSnapshotOperationAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/FinishSnapshotOperationAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/FinishSnapshotOperationAckDiscoveryMessage.java
deleted file mode 100644
index f6758e0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/FinishSnapshotOperationAckDiscoveryMessage.java
+++ /dev/null
@@ -1,84 +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.pagemem.snapshot;
-
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class FinishSnapshotOperationAckDiscoveryMessage implements DiscoveryCustomMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Id. */
-    private final IgniteUuid id = IgniteUuid.randomUuid();
-
-    /** Op id. */
-    private final IgniteUuid opId;
-
-    /** Success. */
-    private final boolean success;
-
-    /**
-     * @param opId Op id.
-     * @param success Success.
-     */
-    public FinishSnapshotOperationAckDiscoveryMessage(IgniteUuid opId, boolean success) {
-        this.opId = opId;
-        this.success = success;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid id() {
-        return id;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isMutable() {
-        return false;
-    }
-
-    /**
-     * @return Op id.
-     */
-    public IgniteUuid operationId() {
-        return opId;
-    }
-
-    /**
-     * @return Success.
-     */
-    public boolean success() {
-        return success;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(FinishSnapshotOperationAckDiscoveryMessage.class, this,
-            "id", id, "opId", opId, "success", success);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotCheckParameters.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotCheckParameters.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotCheckParameters.java
deleted file mode 100644
index 58cb240..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotCheckParameters.java
+++ /dev/null
@@ -1,75 +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.pagemem.snapshot;
-
-import java.io.File;
-import java.io.Serializable;
-import java.util.Collection;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Tuple for passing optional parameters of {@link SnapshotOperationType#CHECK}.
- */
-public class SnapshotCheckParameters implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Optional paths. */
-    private final Collection<File> optionalPaths;
-
-    /** Flag for skipping CRC check. */
-    private final boolean skipCrc;
-
-    /**
-     * Factory method.
-     *
-     * @return Tuple with optional parameters or null if parameters are default.
-     *
-     * @param optionalPaths Optional paths.
-     * @param skipCrc Skip crc.
-     */
-    @Nullable public static SnapshotCheckParameters valueOf(Collection<File> optionalPaths, boolean skipCrc) {
-        if (optionalPaths == null && !skipCrc)
-            return null;
-
-        return new SnapshotCheckParameters(optionalPaths, skipCrc);
-    }
-
-    /**
-     * @param optionalPaths Optional paths.
-     * @param skipCrc Flag for skipping CRC check.
-     */
-    private SnapshotCheckParameters(Collection<File> optionalPaths, boolean skipCrc) {
-        this.optionalPaths = optionalPaths;
-        this.skipCrc = skipCrc;
-    }
-
-    /**
-     * @return Optional paths.
-     */
-    public Collection<File> optionalPaths() {
-        return optionalPaths;
-    }
-
-    /**
-     * @return Flag for skipping CRC check.
-     */
-    public boolean skipCrc() {
-        return skipCrc;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
deleted file mode 100644
index fa18cd7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperation.java
+++ /dev/null
@@ -1,235 +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.pagemem.snapshot;
-
-import java.io.File;
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Description and parameters of snapshot operation
- */
-public class SnapshotOperation implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private final SnapshotOperationType type;
-
-    /** Snapshot ID (the timestamp of snapshot creation). */
-    private final long snapshotId;
-
-    /** Cache group ids. */
-    private final Set<Integer> cacheGrpIds;
-
-    /** Cache names. */
-    private final Set<String> cacheNames;
-
-    /** Message. */
-    private final String msg;
-
-    /** Additional parameter. */
-    private final Object extraParam;
-
-    /** Optional list of dependent snapshot IDs. */
-    private final Set<Long> dependentSnapshotIds;
-
-    /** Optional map of previous snapshots grouped by caches. */
-    private final Map<Long, Set<String>> prevSnapshots;
-
-    /**
-     * @param type Type.
-     * @param snapshotId Snapshot id.
-     * @param cacheGrpIds Cache group ids.
-     * @param cacheNames Cache names.
-     * @param msg Extra user message.
-     * @param extraParam Additional parameter.
-     * @param dependentSnapshotIds Optional list of dependent snapshot IDs.
-     * @param prevSnapshots Optional map of previous snapshots grouped by caches.
-     */
-    public SnapshotOperation(
-        SnapshotOperationType type,
-        long snapshotId,
-        Set<Integer> cacheGrpIds,
-        Set<String> cacheNames,
-        String msg,
-        Object extraParam,
-        Set<Long> dependentSnapshotIds,
-        Map<Long, Set<String>> prevSnapshots
-    ) {
-        this.type = type;
-        this.snapshotId = snapshotId;
-        this.cacheGrpIds = cacheGrpIds;
-        this.cacheNames = cacheNames;
-        this.msg = msg;
-        this.extraParam = extraParam;
-        this.dependentSnapshotIds = dependentSnapshotIds;
-        this.prevSnapshots = prevSnapshots;
-    }
-
-    /**
-     *
-     */
-    public SnapshotOperationType type() {
-        return type;
-    }
-
-    /**
-     * Snapshot ID (the timestamp of snapshot creation).
-     *
-     * @return Snapshot ID.
-     */
-    public long snapshotId() {
-        return snapshotId;
-    }
-
-    /**
-     * Cache group ids included to this snapshot.
-     *
-     * @return Cache names.
-     */
-    public Set<Integer> cacheGroupIds() {
-        return cacheGrpIds;
-    }
-
-    /**
-     * Cache names included to this snapshot.
-     */
-    public Set<String> cacheNames() {
-        return cacheNames;
-    }
-
-    /**
-     * Additional info which was provided by client.
-     */
-    public String message() {
-        return msg;
-    }
-
-    /**
-     *
-     */
-    public Object extraParameter() {
-        return extraParam;
-    }
-
-    /**
-     * @return Optional dependent snapshot IDs.
-     */
-    public Set<Long> dependentSnapshotIds() {
-        return dependentSnapshotIds;
-    }
-
-    /**
-     * @return Cache names grouped by previous snapshot IDs.
-     */
-    public Map<Long, Set<String>> previousSnapshots() {
-        return prevSnapshots;
-    }
-
-    /**
-     * @param op Op.
-     */
-    public static Collection<File> getOptionalPathsParameter(SnapshotOperation op) {
-        assert (op.type() == SnapshotOperationType.RESTORE ||
-            op.type() == SnapshotOperationType.RESTORE_2_PHASE)
-            && (op.extraParameter() == null || op.extraParameter() instanceof Collection)
-            || (op.type() == SnapshotOperationType.CHECK &&
-            (op.extraParameter() == null || op.extraParameter() instanceof SnapshotCheckParameters));
-
-        if (op.type() == SnapshotOperationType.CHECK) {
-            if (op.extraParameter() == null)
-                return null;
-            else
-                return ((SnapshotCheckParameters)op.extraParameter()).optionalPaths();
-        }
-
-        return (Collection<File>)op.extraParameter();
-    }
-
-    /**
-     * @param op Op.
-     */
-    public static boolean getSkipCrcParameter(SnapshotOperation op) {
-        assert op.type() == SnapshotOperationType.CHECK &&
-            (op.extraParameter() == null | op.extraParameter() instanceof SnapshotCheckParameters);
-
-        return op.extraParameter() != null && ((SnapshotCheckParameters)op.extraParameter()).skipCrc();
-    }
-
-    /**
-     * @param op Op.
-     */
-    public static Boolean getFullSnapshotParameter(SnapshotOperation op) {
-        assert op.type() == SnapshotOperationType.CREATE && op.extraParameter() instanceof Boolean;
-
-        return (Boolean)op.extraParameter();
-    }
-
-    /**
-     * @param op Op.
-     */
-    public static File getMovingPathParameter(SnapshotOperation op) {
-        assert op.type() == SnapshotOperationType.MOVE && op.extraParameter() instanceof File;
-
-        return (File)op.extraParameter();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        SnapshotOperation operation = (SnapshotOperation)o;
-
-        if (snapshotId != operation.snapshotId)
-            return false;
-
-        if (type != operation.type)
-            return false;
-
-        return extraParam != null ? extraParam.equals(operation.extraParam) : operation.extraParam == null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = type.hashCode();
-        res = 31 * res + (int)(snapshotId ^ (snapshotId >>> 32));
-        res = 31 * res + (extraParam != null ? extraParam.hashCode() : 0);
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return "SnapshotOperation{" +
-            "type=" + type +
-            ", snapshotId=" + snapshotId +
-            ", cacheNames=" + cacheNames +
-            ", cacheGroupIds=" + cacheGrpIds +
-            ", msg='" + msg + '\'' +
-            ", extraParam=" + extraParam +
-            ", dependentSnapshotIds=" + dependentSnapshotIds +
-            ", prevSnapshots=" + prevSnapshots +
-            '}';
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperationType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperationType.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperationType.java
deleted file mode 100644
index cc1aeea..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/SnapshotOperationType.java
+++ /dev/null
@@ -1,49 +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.pagemem.snapshot;
-
-import org.jetbrains.annotations.Nullable;
-
-/** */
-public enum SnapshotOperationType {
-    /** Create. */
-    CREATE,
-    /** Restore. */
-    RESTORE,
-    /** Restore 2. */
-    RESTORE_2_PHASE,
-    /** Move. */
-    MOVE,
-    /** Delete. */
-    DELETE,
-    /** Check. */
-    CHECK;
-
-    /** Enumerated values. */
-    private static final SnapshotOperationType[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value or {@code null} if ordinal out of range.
-     */
-    @Nullable public static SnapshotOperationType fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationAckDiscoveryMessage.java
deleted file mode 100644
index af7648d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationAckDiscoveryMessage.java
+++ /dev/null
@@ -1,149 +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.pagemem.snapshot;
-
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Message indicating that a snapshot has been started.
- */
-public class StartSnapshotOperationAckDiscoveryMessage implements DiscoveryCustomMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-
-    private SnapshotOperation snapshotOperation;
-
-    /** Custom message ID. */
-    private IgniteUuid id = IgniteUuid.randomUuid();
-
-    /** Operation id. */
-    private IgniteUuid opId;
-
-    /** */
-    private Exception err;
-
-    /** */
-    private UUID initiatorNodeId;
-
-    /** Last full snapshot id for cache. */
-    private Map<Integer, Long> lastFullSnapshotIdForCache;
-
-    /** Last snapshot id for cache. */
-    private Map<Integer, Long> lastSnapshotIdForCache;
-
-    /**
-     * @param snapshotOperation Snapshot Operation.
-     * @param err Error.
-     */
-    public StartSnapshotOperationAckDiscoveryMessage(
-        IgniteUuid id,
-        SnapshotOperation snapshotOperation,
-        Map<Integer, Long> lastFullSnapshotIdForCache,
-        Map<Integer, Long> lastSnapshotIdForCache,
-        Exception err,
-        UUID initiatorNodeId
-    ) {
-        this.opId = id;
-        this.snapshotOperation = snapshotOperation;
-        this.lastFullSnapshotIdForCache = lastFullSnapshotIdForCache;
-        this.lastSnapshotIdForCache = lastSnapshotIdForCache;
-        this.err = err;
-        this.initiatorNodeId = initiatorNodeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid id() {
-        return id;
-    }
-
-    /**
-     *
-     */
-    public boolean needExchange() {
-        /* exchange for trigger saving cluster state*/
-        return err == null && snapshotOperation.type() == SnapshotOperationType.CREATE;
-    }
-
-    /**
-     *
-     */
-    public IgniteUuid operationId() {
-        return opId;
-    }
-
-    /**
-     * @return Initiator node id.
-     */
-    public UUID initiatorNodeId() {
-        return initiatorNodeId;
-    }
-
-    /**
-     * @return Error if start this process is not successfully.
-     */
-    public Exception error() {
-        return err;
-    }
-
-    /**
-     * @return {@code True} if message has error otherwise {@code false}.
-     */
-    public boolean hasError() {
-        return err != null;
-    }
-
-    public SnapshotOperation snapshotOperation() {
-        return snapshotOperation;
-    }
-
-    /**
-     * @param cacheId Cache id.
-     */
-    @Nullable public Long lastFullSnapshotId(int cacheId) {
-        return lastFullSnapshotIdForCache.get(cacheId);
-    }
-
-    /**
-     * @param cacheId Cache id.
-     */
-    @Nullable public Long lastSnapshotId(int cacheId) {
-        return lastSnapshotIdForCache.get(cacheId);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isMutable() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(StartSnapshotOperationAckDiscoveryMessage.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
deleted file mode 100644
index 4c9deb5..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
+++ /dev/null
@@ -1,192 +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.pagemem.snapshot;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Message indicating that a snapshot has been started.
- */
-public class StartSnapshotOperationDiscoveryMessage implements DiscoveryCustomMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Id. */
-    private IgniteUuid id = IgniteUuid.randomUuid();
-
-    /** Custom message ID. */
-    private IgniteUuid operationId;
-
-    /** Snapshot operation. */
-    private SnapshotOperation snapshotOperation;
-
-    /** */
-    private UUID initiatorId;
-
-    /** Validated by coordinator. */
-    private boolean validatedByCoordinator = false;
-
-    /** Error. */
-    private Exception err;
-
-    /** Last full snapshot id for cache. */
-    private Map<Integer, Long> lastFullSnapshotIdForCache = new HashMap<>();
-
-    /** Last snapshot id for cache. */
-    private Map<Integer, Long> lastSnapshotIdForCache = new HashMap<>();
-
-    /**
-     * @param snapshotOperation Snapshot operation
-     * @param initiatorId initiator node id
-     */
-    public StartSnapshotOperationDiscoveryMessage(
-        IgniteUuid operationId,
-        SnapshotOperation snapshotOperation,
-        UUID initiatorId
-    ) {
-        this.operationId = operationId;
-        this.snapshotOperation = snapshotOperation;
-        this.initiatorId = initiatorId;
-    }
-
-    /**
-     *
-     */
-    public SnapshotOperation snapshotOperation() {
-        return snapshotOperation;
-    }
-
-    /**
-     * Sets error.
-     *
-     * @param err Error.
-     */
-    public void error(Exception err) {
-        this.err = err;
-    }
-
-    /**
-     * @return {@code True} if message contains error.
-     */
-    public boolean hasError() {
-        return err != null;
-    }
-
-    /**
-     * @return Error.
-     */
-    public Exception error() {
-        return err;
-    }
-
-    /**
-     * @return Initiator node id.
-     */
-    public UUID initiatorNodeId() {
-        return initiatorId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteUuid id() {
-        return id;
-    }
-
-    /**
-     * @return Operation ID.
-     */
-    public IgniteUuid operationId() {
-        return operationId;
-    }
-
-    /**
-     * @param cacheId Cache id.
-     */
-    public Long lastFullSnapshotId(int cacheId) {
-        return lastFullSnapshotIdForCache.get(cacheId);
-    }
-
-    /**
-     * @param cacheId Cache id.
-     * @param id Id.
-     */
-    public void lastFullSnapshotId(int cacheId, long id) {
-        lastFullSnapshotIdForCache.put(cacheId, id);
-    }
-
-    /**
-     * @param cacheId Cache id.
-     */
-    public Long lastSnapshotId(int cacheId) {
-        return lastSnapshotIdForCache.get(cacheId);
-    }
-
-    /** @return Validated by coordinator. */
-    public boolean validatedByCoordinator() {
-        return validatedByCoordinator;
-    }
-
-    /** Validated by coordinator. */
-    public void validatedByCoordinator(boolean validatedByCoordinator) {
-        this.validatedByCoordinator = validatedByCoordinator;
-    }
-
-    /**
-     * @param cacheId Cache id.
-     * @param id Id.
-     */
-    public void lastSnapshotId(int cacheId, long id) {
-        lastSnapshotIdForCache.put(cacheId, id);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
-        return new StartSnapshotOperationAckDiscoveryMessage(
-            operationId,
-            snapshotOperation,
-            lastFullSnapshotIdForCache,
-            lastSnapshotIdForCache,
-            err != null ? err : (validatedByCoordinator? null : new IgniteException("Coordinator didn't validate operation!")),
-            initiatorId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isMutable() {
-        return true;
-    }
-
-    /**
-     * @param snapshotOperation new snapshot operation
-     */
-    public void snapshotOperation(SnapshotOperation snapshotOperation) {
-        this.snapshotOperation = snapshotOperation;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(StartSnapshotOperationDiscoveryMessage.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
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 8506cde..f1db79a 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
@@ -32,7 +32,6 @@ import java.util.Map;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.LinkedBlockingDeque;
@@ -61,7 +60,6 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData;
 import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
-import org.apache.ignite.internal.pagemem.snapshot.StartSnapshotOperationAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
@@ -79,6 +77,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionsToReloadMap;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -87,8 +86,8 @@ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchangeWorkerTask;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridListSet;
-import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.GridPartitionStateMap;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -421,8 +420,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     exchangeFuture(msg.exchangeId(), null, null, null, null)
                         .onAffinityChangeMessage(evt.eventNode(), msg);
             }
-            else if (customMsg instanceof StartSnapshotOperationAckDiscoveryMessage
-                && ((StartSnapshotOperationAckDiscoveryMessage)customMsg).needExchange()) {
+            else if (customMsg instanceof SnapshotDiscoveryMessage
+                && ((SnapshotDiscoveryMessage) customMsg).needExchange()) {
                 exchId = exchangeId(n.id(), affinityTopologyVersion(evt), evt.type());
 
                 exchFut = exchangeFuture(exchId, evt, null, null, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
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 0a69d72..0488a14 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
@@ -69,21 +69,13 @@ import org.apache.ignite.internal.binary.BinaryContext;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.pagemem.snapshot.StartSnapshotOperationAckDiscoveryMessage;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.CacheJoinNodeDiscoveryData.CacheInfo;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
+import org.apache.ignite.internal.processors.cache.CacheJoinNodeDiscoveryData.CacheInfo;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
-import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheSnapshotManager;
-import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
-import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
-import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
@@ -97,6 +89,14 @@ 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.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
+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;
@@ -2969,8 +2969,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (msg instanceof CacheAffinityChangeMessage)
             return sharedCtx.affinity().onCustomEvent(((CacheAffinityChangeMessage)msg));
 
-        if (msg instanceof StartSnapshotOperationAckDiscoveryMessage &&
-            ((StartSnapshotOperationAckDiscoveryMessage)msg).needExchange())
+        if (msg instanceof SnapshotDiscoveryMessage &&
+            ((SnapshotDiscoveryMessage)msg).needExchange())
             return true;
 
         if (msg instanceof DynamicCacheChangeBatch)

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 40b263f..efd90a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -42,12 +42,12 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 2f54810..a69872f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1740,11 +1740,11 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                         result.add(e.getKey());
                     }
-                }
 
-                U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
-                    "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
-                    ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]");
+                    U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
+                        "[nodeId=" + ctx.localNodeId() + "cacheOrGroupName=" + grp.cacheOrGroupName() +
+                        ", partId=" + p + ", haveHistory=" + haveHistory + "]");
+                }
             }
 
             if (updateSeq)

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 97fcb12..90c8aaf 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
@@ -52,8 +52,6 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
-import org.apache.ignite.internal.pagemem.snapshot.SnapshotOperation;
-import org.apache.ignite.internal.pagemem.snapshot.StartSnapshotOperationAckDiscoveryMessage;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage;
@@ -66,11 +64,12 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.StateChangeRequest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage;
@@ -581,7 +580,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
 
                     exchange = onCacheChangeRequest(crdNode);
                 }
-                else if (msg instanceof StartSnapshotOperationAckDiscoveryMessage) {
+                else if (msg instanceof SnapshotDiscoveryMessage) {
                     exchange = CU.clientNode(discoEvt.eventNode()) ?
                         onClientNodeEvent(crdNode) :
                         onServerNodeEvent(crdNode);
@@ -658,7 +657,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             }
 
             if (cctx.localNode().isClient())
-                startLocalSnasphotOperation();
+                tryToPerformLocalSnapshotOperation();
 
             exchLog.info("Finished exchange init [topVer=" + topVer + ", crd=" + crdNode + ']');
         }
@@ -1007,26 +1006,18 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-
+     * Try to start local snapshot operation if it is needed by discovery event
      */
-    private void startLocalSnasphotOperation() {
-        StartSnapshotOperationAckDiscoveryMessage snapOpMsg= getSnapshotOperationMessage();
-
-        if (snapOpMsg != null) {
-            SnapshotOperation op = snapOpMsg.snapshotOperation();
-
-            assert snapOpMsg.needExchange();
-
-            try {
-                IgniteInternalFuture fut = cctx.snapshot()
-                    .startLocalSnapshotOperation(snapOpMsg.initiatorNodeId(), snapOpMsg.snapshotOperation());
+    private void tryToPerformLocalSnapshotOperation() {
+        try {
+            IgniteInternalFuture fut = cctx.snapshot()
+                .tryStartLocalSnapshotOperation(discoEvt);
 
-                if (fut != null)
-                    fut.get();
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Error while starting snapshot operation", e);
-            }
+            if (fut != null)
+                fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Error while starting snapshot operation", e);
         }
     }
 
@@ -1395,7 +1386,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             grpValidRes = m;
         }
 
-        startLocalSnasphotOperation();
+        tryToPerformLocalSnapshotOperation();
 
         cctx.cache().onExchangeDone(exchId.topologyVersion(), exchActions, err);
 
@@ -1456,21 +1447,6 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     *
-     */
-    private StartSnapshotOperationAckDiscoveryMessage getSnapshotOperationMessage() {
-        // If it's a snapshot operation request, synchronously wait for backup start.
-        if (discoEvt != null && discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) {
-            DiscoveryCustomMessage customMsg = ((DiscoveryCustomEvent)discoEvt).customMessage();
-
-            if (customMsg instanceof StartSnapshotOperationAckDiscoveryMessage)
-                return  (StartSnapshotOperationAckDiscoveryMessage)customMsg;
-        }
-
-        return null;
-    }
-
-    /**
      * Cleans up resources to avoid excessive memory usage.
      */
     public void cleanUp() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
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 d64677e..8fe9377 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
@@ -81,7 +81,6 @@ import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
-import org.apache.ignite.internal.pagemem.snapshot.SnapshotOperation;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.pagemem.wal.StorageException;
@@ -110,6 +109,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStor
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
deleted file mode 100644
index cce6f55..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
+++ /dev/null
@@ -1,150 +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.persistence;
-
-import java.nio.ByteBuffer;
-import java.util.NavigableMap;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.pagemem.snapshot.SnapshotOperation;
-import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
-import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
-import org.apache.ignite.internal.util.typedef.T2;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class IgniteCacheSnapshotManager extends GridCacheSharedManagerAdapter implements IgniteChangeGlobalStateSupport {
-    /** Snapshot started lock filename. */
-    public static final String SNAPSHOT_RESTORE_STARTED_LOCK_FILENAME = "snapshot-started.loc";
-
-    /**
-     * @param initiatorNodeId Initiator node id.
-     * @param snapshotOperation Snapshot operation.
-     */
-    @Nullable public IgniteInternalFuture startLocalSnapshotOperation(
-        UUID initiatorNodeId,
-        SnapshotOperation snapshotOperation
-    ) throws IgniteCheckedException {
-        return null;
-    }
-
-    /**
-     * @param snapOp current snapshot operation.
-     *
-     * @return {@code true} if next operation must be snapshot, {@code false} if checkpoint must be executed.
-     */
-    public boolean onMarkCheckPointBegin(
-        SnapshotOperation snapOp,
-        NavigableMap<T2<Integer, Integer>, T2<Integer, Integer>> map
-    ) throws IgniteCheckedException {
-        return false;
-    }
-
-    /**
-     *
-     */
-    public void restoreState() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    public boolean snapshotOperationInProgress(){
-        return false;
-    }
-
-    /**
-     *
-     */
-    public void beforeCheckpointPageWritten() {
-        // No-op.
-    }
-
-    /**
-     *
-     */
-    public void afterCheckpointPageWritten() {
-        // No-op.
-    }
-
-    /**
-     * @param fullId Full id.
-     */
-    public void beforePageWrite(FullPageId fullId) {
-        // No-op.
-    }
-
-    /**
-     * @param fullId Full id.
-     */
-    public void onPageWrite(FullPageId fullId, ByteBuffer tmpWriteBuf) {
-        // No-op.
-    }
-
-    /**
-     * @param cctx Cctx.
-     */
-    public void onCacheStop(GridCacheContext cctx) {
-        // No-op.
-    }
-
-    /**
-     * @param gctx Cctx.
-     */
-    public void onCacheGroupStop(CacheGroupContext gctx) {
-        // No-op.
-    }
-
-    /**
-     *
-     */
-    public void onChangeTrackerPage(
-        Long page,
-        FullPageId fullId,
-        PageMemory pageMem
-    ) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /**
-     *
-     */
-    public void flushDirtyPageHandler(
-        FullPageId fullId,
-        ByteBuffer pageBuf,
-        Integer tag
-    ) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index 28bf6e4..6aa2243 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -47,7 +47,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheSnapshotManager;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
new file mode 100644
index 0000000..0a27bcd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
@@ -0,0 +1,161 @@
+/*
+ * 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.persistence.snapshot;
+
+import java.nio.ByteBuffer;
+import java.util.NavigableMap;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
+import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Snapshot manager stub.
+ */
+public class IgniteCacheSnapshotManager<T extends SnapshotOperation> extends GridCacheSharedManagerAdapter implements IgniteChangeGlobalStateSupport {
+    /** Snapshot started lock filename. */
+    public static final String SNAPSHOT_RESTORE_STARTED_LOCK_FILENAME = "snapshot-started.loc";
+
+    /**
+     * Try to start local snapshot operation if it's required by discovery event.
+     *
+     * @param discoveryEvent Discovery event.
+     */
+    @Nullable public IgniteInternalFuture tryStartLocalSnapshotOperation(
+            @Nullable DiscoveryEvent discoveryEvent
+    ) throws IgniteCheckedException {
+        return null;
+    }
+
+    /**
+     * @param initiatorNodeId Initiator node id.
+     * @param snapshotOperation Snapshot operation.
+     */
+    @Nullable public IgniteInternalFuture startLocalSnapshotOperation(
+        UUID initiatorNodeId,
+        T snapshotOperation
+    ) throws IgniteCheckedException {
+        return null;
+    }
+
+    /**
+     * @param snapshotOperation current snapshot operation.
+     *
+     * @return {@code true} if next operation must be snapshot, {@code false} if checkpoint must be executed.
+     */
+    public boolean onMarkCheckPointBegin(
+        T snapshotOperation,
+        NavigableMap<T2<Integer, Integer>, T2<Integer, Integer>> map
+    ) throws IgniteCheckedException {
+        return false;
+    }
+
+    /**
+     *
+     */
+    public void restoreState() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    public boolean snapshotOperationInProgress(){
+        return false;
+    }
+
+    /**
+     *
+     */
+    public void beforeCheckpointPageWritten() {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    public void afterCheckpointPageWritten() {
+        // No-op.
+    }
+
+    /**
+     * @param fullId Full id.
+     */
+    public void beforePageWrite(FullPageId fullId) {
+        // No-op.
+    }
+
+    /**
+     * @param fullId Full id.
+     */
+    public void onPageWrite(FullPageId fullId, ByteBuffer tmpWriteBuf) {
+        // No-op.
+    }
+
+    /**
+     * @param cctx Cctx.
+     */
+    public void onCacheStop(GridCacheContext cctx) {
+        // No-op.
+    }
+
+    /**
+     * @param gctx Cctx.
+     */
+    public void onCacheGroupStop(CacheGroupContext gctx) {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    public void onChangeTrackerPage(
+        Long page,
+        FullPageId fullId,
+        PageMemory pageMem
+    ) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    public void flushDirtyPageHandler(
+        FullPageId fullId,
+        ByteBuffer pageBuf,
+        Integer tag
+    ) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDeActivate(GridKernalContext kctx) {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotDiscoveryMessage.java
new file mode 100644
index 0000000..d88d96e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotDiscoveryMessage.java
@@ -0,0 +1,33 @@
+/*
+ * 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.persistence.snapshot;
+
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+
+/**
+ * Initial snapshot discovery message with possibility to trigger exchange.
+ */
+public interface SnapshotDiscoveryMessage extends DiscoveryCustomMessage {
+    /**
+     * Is exchange needed after receiving this message.
+     *
+     * @return True if exchange is needed, false in other case.
+     */
+    boolean needExchange();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperation.java
new file mode 100644
index 0000000..6722eb6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperation.java
@@ -0,0 +1,44 @@
+/*
+ * 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.persistence.snapshot;
+
+import java.io.Serializable;
+import java.util.Set;
+
+/**
+ * Initial snapshot operation interface.
+ */
+public interface SnapshotOperation extends Serializable {
+    /**
+     * Cache group ids included to this snapshot.
+     *
+     * @return Cache names.
+     */
+    Set<Integer> cacheGroupIds();
+
+    /**
+     * Cache names included to this snapshot.
+     */
+    Set<String> cacheNames();
+
+    /**
+     * Any custom extra parameter.
+     */
+    Object extraParameter();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index e34a772..8c0f400 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -337,12 +337,7 @@ org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager$1
 org.apache.ignite.internal.marshaller.optimized.OptimizedFieldType
 org.apache.ignite.internal.mem.IgniteOutOfMemoryException
 org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl$Segment
-org.apache.ignite.internal.pagemem.snapshot.FinishSnapshotOperationAckDiscoveryMessage
 org.apache.ignite.internal.pagemem.snapshot.SnapshotCheckParameters
-org.apache.ignite.internal.pagemem.snapshot.SnapshotOperation
-org.apache.ignite.internal.pagemem.snapshot.SnapshotOperationType
-org.apache.ignite.internal.pagemem.snapshot.StartSnapshotOperationAckDiscoveryMessage
-org.apache.ignite.internal.pagemem.snapshot.StartSnapshotOperationDiscoveryMessage
 org.apache.ignite.internal.pagemem.wal.StorageException
 org.apache.ignite.internal.pagemem.wal.WALIterator
 org.apache.ignite.internal.pagemem.wal.record.TxRecord$TxAction

http://git-wip-us.apache.org/repos/asf/ignite/blob/36716fb1/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 12200ae..6a1d4f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -33,13 +33,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheIoManager;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccManager;
 import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.GridCacheTtlManager;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedTtlCleanupManager;
-import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheSnapshotManager;
+import org.apache.ignite.internal.processors.cache.GridCacheTtlManager;
 import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager;
 import org.apache.ignite.internal.processors.cache.dr.GridOsCacheDrManager;
 import org.apache.ignite.internal.processors.cache.jta.CacheNoopJtaManager;
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.query.GridCacheLocalQueryManager;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
 import org.apache.ignite.internal.processors.cache.store.CacheOsStoreManager;


[17/41] ignite git commit: IGNITE-5204: SQL: fixed incorrect partition calculation in case of unicast optimization, when WHERE clause argument type was different from key or affinity key type. This close #2107.

Posted by sb...@apache.org.
IGNITE-5204: SQL: fixed incorrect partition calculation in case of unicast optimization, when WHERE clause argument type was different from key or affinity key type. This close #2107.


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

Branch: refs/heads/ignite-5578-1
Commit: 99713feea764fc8c3e5b247a24698a2c04d3bcf4
Parents: 55a5ca0
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Fri Jul 7 16:30:37 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 7 16:30:37 2017 +0300

----------------------------------------------------------------------
 .../cache/query/CacheQueryPartitionInfo.java    |  46 +++-
 .../query/h2/DmlStatementsProcessor.java        |   9 +-
 .../internal/processors/query/h2/H2Utils.java   |  27 +++
 .../processors/query/h2/IgniteH2Indexing.java   |  30 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   5 +-
 .../processors/query/IgniteSqlRoutingTest.java  | 211 +++++++++++++++++--
 6 files changed, 290 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
index 1329d5c..42bf070 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPartitionInfo.java
@@ -39,22 +39,35 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  */
 public class CacheQueryPartitionInfo {
     /** */
-    private int partId;
+    private final int partId;
 
     /** */
-    private String cacheName;
+    private final String cacheName;
 
     /** */
-    private int paramIdx;
+    private final String tableName;
+
+    /** */
+    private final int dataType;
+
+    /** */
+    private final int paramIdx;
 
     /**
      * @param partId Partition id, or -1 if parameter binding required.
      * @param cacheName Cache name required for partition calculation.
+     * @param tableName Table name required for proper type conversion.
+     * @param dataType Required data type id for the query parameter.
      * @param paramIdx Query parameter index required for partition calculation.
      */
-    public CacheQueryPartitionInfo(int partId, String cacheName, int paramIdx) {
+    public CacheQueryPartitionInfo(int partId, String cacheName, String tableName, int dataType, int paramIdx) {
+        // In case partition is not known, both cacheName and tableName must be provided.
+        assert (partId >= 0) ^ ((cacheName != null) && (tableName != null));
+
         this.partId = partId;
         this.cacheName = cacheName;
+        this.tableName = tableName;
+        this.dataType = dataType;
         this.paramIdx = paramIdx;
     }
 
@@ -73,6 +86,20 @@ public class CacheQueryPartitionInfo {
     }
 
     /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tableName;
+    }
+
+    /**
+     * @return Required data type for the query parameter.
+     */
+    public int dataType() {
+        return dataType;
+    }
+
+    /**
      * @return Query parameter index required for partition calculation.
      */
     public int paramIdx() {
@@ -81,7 +108,9 @@ public class CacheQueryPartitionInfo {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return partId ^ paramIdx ^ (cacheName == null ? 0 : cacheName.hashCode());
+        return partId ^ dataType ^ paramIdx ^
+            (cacheName == null ? 0 : cacheName.hashCode()) ^
+            (tableName == null ? 0 : tableName.hashCode());
     }
 
     /** {@inheritDoc} */
@@ -97,10 +126,13 @@ public class CacheQueryPartitionInfo {
         if (partId >= 0)
             return partId == other.partId;
 
-        if (other.cacheName == null)
+        if (other.cacheName == null || other.tableName == null)
             return false;
 
-        return other.cacheName.equals(cacheName) && other.paramIdx == paramIdx;
+        return other.cacheName.equals(cacheName) &&
+            other.tableName.equals(tableName) &&
+            other.dataType == dataType &&
+            other.paramIdx == paramIdx;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 0c1dbf9..4f7c288 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -748,14 +748,7 @@ public class DmlStatementsProcessor {
             return newArr;
         }
 
-        int objType = DataType.getTypeFromClass(val.getClass());
-
-        if (objType == type)
-            return val;
-
-        Value h2Val = desc.wrap(val, objType);
-
-        return h2Val.convertTo(type).getObject();
+        return H2Utils.convert(val, desc, type);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
index ee88acf..157e1ba 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
@@ -29,6 +30,8 @@ import org.h2.engine.Session;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.result.SortOrder;
 import org.h2.table.IndexColumn;
+import org.h2.value.DataType;
+import org.h2.value.Value;
 
 import java.lang.reflect.Constructor;
 import java.sql.Connection;
@@ -236,6 +239,30 @@ public class H2Utils {
     }
 
     /**
+     * Convert value to column's expected type by means of H2.
+     *
+     * @param val Source value.
+     * @param desc Row descriptor.
+     * @param type Expected column type to convert to.
+     * @return Converted object.
+     * @throws IgniteCheckedException if failed.
+     */
+    public static Object convert(Object val, GridH2RowDescriptor desc, int type)
+        throws IgniteCheckedException {
+        if (val == null)
+            return null;
+
+        int objType = DataType.getTypeFromClass(val.getClass());
+
+        if (objType == type)
+            return val;
+
+        Value h2Val = desc.wrap(val, objType);
+
+        return h2Val.convertTo(type).getObject();
+    }
+
+    /**
      * Private constructor.
      */
     private H2Utils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 255c5f1..40eae17 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -2265,7 +2265,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /**
      * Bind query parameters and calculate partitions derived from the query.
      *
+     * @param partInfoList Collection of query derived partition info.
+     * @param params Query parameters.
      * @return Partitions.
+     * @throws IgniteCheckedException, If fails.
      */
     private int[] calculateQueryPartitions(CacheQueryPartitionInfo[] partInfoList, Object[] params)
         throws IgniteCheckedException {
@@ -2273,9 +2276,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         ArrayList<Integer> list = new ArrayList<>(partInfoList.length);
 
         for (CacheQueryPartitionInfo partInfo: partInfoList) {
-            int partId = partInfo.partition() < 0 ?
-                kernalContext().affinity().partition(partInfo.cacheName(), params[partInfo.paramIdx()]) :
-                partInfo.partition();
+            int partId = (partInfo.partition() >= 0) ? partInfo.partition() :
+                bindPartitionInfoParameter(partInfo, params);
 
             int i = 0;
 
@@ -2298,6 +2300,28 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return result;
     }
 
+    /**
+     * Bind query parameter to partition info and calculate partition.
+     *
+     * @param partInfo Partition Info.
+     * @param params Query parameters.
+     * @return Partition.
+     * @throws IgniteCheckedException, If fails.
+     */
+    private int bindPartitionInfoParameter(CacheQueryPartitionInfo partInfo, Object[] params)
+        throws IgniteCheckedException {
+        assert partInfo != null;
+        assert partInfo.partition() < 0;
+
+        GridH2RowDescriptor desc = dataTable(partInfo.cacheName(),
+                partInfo.tableName()).rowDescriptor();
+
+        Object param = H2Utils.convert(params[partInfo.paramIdx()],
+                desc, partInfo.dataType());
+
+        return kernalContext().affinity().partition(partInfo.cacheName(), param);
+    }
+
     /** {@inheritDoc} */
     @Override public Collection<GridRunningQueryInfo> runningQueries(long duration) {
         Collection<GridRunningQueryInfo> res = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index aebf596..1578f9f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -2318,14 +2318,15 @@ public class GridSqlQuerySplitter {
             GridSqlConst constant = (GridSqlConst)right;
 
             return new CacheQueryPartitionInfo(ctx.affinity().partition(tbl.cacheName(),
-                constant.value().getObject()), null, -1);
+                constant.value().getObject()), null, null, -1, -1);
         }
 
         assert right instanceof GridSqlParameter;
 
         GridSqlParameter param = (GridSqlParameter) right;
 
-        return new CacheQueryPartitionInfo(-1, tbl.cacheName(), param.index());
+        return new CacheQueryPartitionInfo(-1, tbl.cacheName(), tbl.getName(),
+            column.column().getType(), param.index());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/99713fee/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
index fddd3f4..323eb7a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlRoutingTest.java
@@ -33,19 +33,26 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.text.DateFormat;
 import java.text.MessageFormat;
+import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Date;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
 
+/** Tests for query partitions derivation. */
 public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
@@ -62,10 +69,10 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
     /** */
     private static int NODE_COUNT = 4;
 
-    /** broadcast query to ensure events came from all nodes */
+    /** Broadcast query to ensure events came from all nodes. */
     private static String FINAL_QRY = "select count(1) from {0} where name=?";
 
-    /** Param to distinguish the final query event */
+    /** Param to distinguish the final query event. */
     private static String FINAL_QRY_PARAM = "Abracadabra";
 
     /** {@inheritDoc} */
@@ -108,15 +115,19 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
 
         startGrid(NODE_CLIENT);
 
+        awaitPartitionMapExchange();
+
         fillCaches();
     }
 
+    /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         super.afterTestsStopped();
 
         stopAllGrids();
     }
 
+    /** */
     private CacheConfiguration buildCacheConfiguration(String name) {
         if (name.equals(CACHE_PERSON)) {
             CacheConfiguration ccfg = new CacheConfiguration(CACHE_PERSON);
@@ -176,7 +187,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select id, name, duration from Call where personId=100 order by id"), 1);
+            new SqlFieldsQuery("select id, name, duration from Call where personId=100 order by id"), 1);
 
         assertEquals(2, result.size());
 
@@ -189,7 +200,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select id, name, duration from Call where personId=? order by id").setArgs(100), 1);
+            new SqlFieldsQuery("select id, name, duration from Call where personId=? order by id")
+            .setArgs(100), 1);
 
         assertEquals(2, result.size());
 
@@ -203,7 +215,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
 
         for (int key : new int[] {0, 250, 500, 750, 1000} ) {
             List<List<?>> result = runQueryEnsureUnicast(cache,
-                    new SqlFieldsQuery("select name, age from Person where _key=?").setArgs(key), 1);
+                new SqlFieldsQuery("select name, age from Person where _key=?").setArgs(key), 1);
 
             assertEquals(1, result.size());
 
@@ -220,8 +232,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         CallKey callKey = new CallKey(5, 1);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select name, duration from Call where _key=?")
-                .setArgs(callKey), 1);
+            new SqlFieldsQuery("select name, duration from Call where _key=?")
+            .setArgs(callKey), 1);
 
         assertEquals(1, result.size());
 
@@ -230,7 +242,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         checkResultsRow(result, 0, call.name, call.duration);
     }
 
-    /** Check group, having, ordering allowed to be unicast requests */
+    /** Check group, having, ordering allowed to be unicast requests. */
     public void testUnicastQueryGroups() throws Exception {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
@@ -258,8 +270,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         CallKey callKey = new CallKey(5, 1);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select name, duration from Call where _key=? and duration=?")
-                        .setArgs(callKey, 100), 1);
+            new SqlFieldsQuery("select name, duration from Call where _key=? and duration=?")
+            .setArgs(callKey, 100), 1);
 
         assertEquals(1, result.size());
 
@@ -276,8 +288,8 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         CallKey callKey2 = new CallKey(1000, 1);
 
         List<List<?>> result = runQueryEnsureUnicast(cache,
-                new SqlFieldsQuery("select name, duration from Call where (_key=? and duration=?) or (_key=?)")
-                        .setArgs(callKey1, 100, callKey2), 2);
+            new SqlFieldsQuery("select name, duration from Call where (_key=? and duration=?) or (_key=?)")
+            .setArgs(callKey1, 100, callKey2), 2);
 
         assertEquals(2, result.size());
 
@@ -291,19 +303,181 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
     }
 
     /** */
+    public void testUnicastQueryKeyTypeConversionParameter() throws Exception {
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON);
+
+        // Pass string argument to expression with integer
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select name, age from Person where _key = ?")
+            .setArgs("5"), 1);
+
+        Person person = cache.get(5);
+
+        assertEquals(1, result.size());
+
+        assertEquals(person.name, result.get(0).get(0));
+        assertEquals(person.age, result.get(0).get(1));
+    }
+
+    /** */
+    public void testUnicastQueryKeyTypeConversionConstant() throws Exception {
+        IgniteCache<Integer, Person> cache = grid(NODE_CLIENT).cache(CACHE_PERSON);
+
+        // Use string within expression against integer key
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select name, age from Person where _key = '5'"), 1);
+
+        Person person = cache.get(5);
+
+        assertEquals(1, result.size());
+
+        assertEquals(person.name, result.get(0).get(0));
+        assertEquals(person.age, result.get(0).get(1));
+    }
+
+    /** */
+    public void testUnicastQueryAffinityKeyTypeConversionParameter() throws Exception {
+        IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
+
+        // Pass string argument to expression with integer
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select id, name, duration from Call where personId=? order by id")
+                .setArgs("100"), 1);
+
+        assertEquals(2, result.size());
+
+        checkResultsRow(result, 0, 1, "caller1", 100);
+        checkResultsRow(result, 1, 2, "caller2", 200);
+    }
+
+    /** */
+    public void testUnicastQueryAffinityKeyTypeConversionConstant() throws Exception {
+        IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
+
+        // Use string within expression against integer key
+        List<List<?>> result = runQueryEnsureUnicast(cache,
+            new SqlFieldsQuery("select id, name, duration from Call where personId='100' order by id"), 1);
+
+        assertEquals(2, result.size());
+
+        checkResultsRow(result, 0, 1, "caller1", 100);
+        checkResultsRow(result, 1, 2, "caller2", 200);
+    }
+
+    /** */
     public void testBroadcastQuerySelectKeyEqualsOrFieldParameter() throws Exception {
         IgniteCache<CallKey, Call> cache = grid(NODE_CLIENT).cache(CACHE_CALL);
 
         CallKey callKey = new CallKey(5, 1);
 
         List<List<?>> result = runQueryEnsureBroadcast(cache,
-                new SqlFieldsQuery("select name, duration from Call where _key=? or duration=?")
-                        .setArgs(callKey, 100));
+            new SqlFieldsQuery("select name, duration from Call where _key=? or duration=?")
+            .setArgs(callKey, 100));
 
         assertEquals(cache.size() / 2, result.size());
     }
 
     /** */
+    public void testUuidKeyAsByteArrayParameter() throws Exception {
+        String cacheName = "uuidCache";
+
+        CacheConfiguration<UUID, UUID> ccfg = new CacheConfiguration<>(cacheName);
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+
+        ccfg.setIndexedTypes(UUID.class, UUID.class);
+
+        IgniteCache<UUID, UUID> cache = grid(NODE_CLIENT).createCache(ccfg);
+
+        try {
+            int count = 10;
+
+            UUID values[] = new UUID[count];
+
+            for (int i = 0; i < count; i++) {
+                UUID val = UUID.randomUUID();
+
+                cache.put(val, val);
+
+                values[i] = val;
+            }
+
+            for (UUID val : values) {
+                byte[] arr = convertUuidToByteArray(val);
+
+                List<List<?>> result = cache.query(new SqlFieldsQuery(
+                    "select _val from UUID where _key = ?").setArgs(arr)).getAll();
+
+                assertEquals(1, result.size());
+                assertEquals(val, result.get(0).get(0));
+            }
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /** */
+    public void testDateKeyAsTimestampParameter() throws Exception {
+        String cacheName = "dateCache";
+
+        CacheConfiguration<Date, Date> ccfg = new CacheConfiguration<>(cacheName);
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+
+        ccfg.setIndexedTypes(Date.class, Date.class);
+
+        IgniteCache<Date, Date> cache = grid(NODE_CLIENT).createCache(ccfg);
+
+        try {
+            int count = 30;
+
+            Date values[] = new Date[count];
+
+            DateFormat dateFormat = new SimpleDateFormat("dd/MM/yyyy");
+
+            for (int i = 0; i < count; i++) {
+                Date val = dateFormat.parse(String.format("%02d/06/2017", i + 1));
+
+                cache.put(val, val);
+
+                values[i] = val;
+            }
+
+            for (Date val : values) {
+                Timestamp ts = new Timestamp(val.getTime());
+
+                List<List<?>> result = cache.query(new SqlFieldsQuery(
+                    "select _val from Date where _key = ?").setArgs(ts)).getAll();
+
+                assertEquals(1, result.size());
+                assertEquals(val, result.get(0).get(0));
+            }
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * Convert UUID to byte[].
+     *
+     * @param val UUID to convert.
+     * @return Result.
+     */
+    private byte[] convertUuidToByteArray(UUID val) {
+        assert val != null;
+
+        ByteBuffer bb = ByteBuffer.wrap(new byte[16]);
+
+        bb.putLong(val.getMostSignificantBits());
+
+        bb.putLong(val.getLeastSignificantBits());
+
+        return bb.array();
+    }
+
+    /** */
     private void fillCaches() {
         IgniteCache<CallKey, Call> callCache = grid(NODE_CLIENT).cache(CACHE_CALL);
         IgniteCache<Integer, Person> personCache = grid(NODE_CLIENT).cache(CACHE_PERSON);
@@ -335,15 +509,15 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
             assertEquals(expected[col], row.get(col));
     }
 
-    /** Run query and check that only one node did generate 'query executed' event for it */
+    /** Run query and check that only one node did generate 'query executed' event for it. */
     private List<List<?>> runQueryEnsureUnicast(IgniteCache<?,?> cache, SqlFieldsQuery qry, int nodeCnt) throws Exception {
         try (EventCounter evtCounter = new EventCounter(nodeCnt)) {
             List<List<?>> result = cache.query(qry).getAll();
 
             // do broadcast 'marker' query to ensure that we received all events from previous qry
             cache.query(new SqlFieldsQuery(
-                    MessageFormat.format(FINAL_QRY, cache.getName()))
-                    .setArgs(FINAL_QRY_PARAM)).getAll();
+                MessageFormat.format(FINAL_QRY, cache.getName()))
+                .setArgs(FINAL_QRY_PARAM)).getAll();
 
             // wait for all events from 'marker' query
             evtCounter.await();
@@ -353,6 +527,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         }
     }
 
+    /** */
     private List<List<?>> runQueryEnsureBroadcast(IgniteCache<?, ?> cache, SqlFieldsQuery qry) throws Exception {
         final CountDownLatch execLatch = new CountDownLatch(NODE_COUNT);
 
@@ -492,7 +667,7 @@ public class IgniteSqlRoutingTest extends GridCommonAbstractTest {
         private int id;
 
         /** */
-        public CallKey(int personId, int id) {
+        private CallKey(int personId, int id) {
             this.personId = personId;
             this.id = id;
         }


[11/41] ignite git commit: IGNITE-5717 .NET: Reduce MemoryPolicyConfiguration.MaxSize for persistence tests as a workaround for OOM on default settings

Posted by sb...@apache.org.
IGNITE-5717 .NET: Reduce MemoryPolicyConfiguration.MaxSize for persistence tests as a workaround for OOM on default settings


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

Branch: refs/heads/ignite-5578-1
Commit: a9387adef490086406b05fe961ff0f3151045caa
Parents: d232648
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jul 7 15:21:20 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jul 7 15:21:20 2017 +0300

----------------------------------------------------------------------
 .../Cache/PersistentStoreTest.cs                | 36 ++++++++++++++++++--
 1 file changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9387ade/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
index 96ae47c..e9cbce8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Tests.Cache
 {
     using System.IO;
+    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.PersistentStore;
@@ -51,7 +52,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         [Test]
         public void TestCacheDataSurvivesNodeRestart()
         {
-            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            var cfg = new IgniteConfiguration(GetTestConfiguration())
             {
                 PersistentStoreConfiguration = new PersistentStoreConfiguration
                 {
@@ -106,7 +107,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         [Test]
         public void TestGridActivationWithPersistence()
         {
-            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            var cfg = new IgniteConfiguration(GetTestConfiguration())
             {
                 PersistentStoreConfiguration = new PersistentStoreConfiguration()
             };
@@ -177,5 +178,36 @@ namespace Apache.Ignite.Core.Tests.Cache
                     ex.Message.Substring(0, 62));
             }
         }
+
+        /// <summary>
+        /// Gets the test configuration.
+        /// </summary>
+        private static IgniteConfiguration GetTestConfiguration()
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                MemoryConfiguration = GetMemoryConfig()
+            };
+        }
+
+        /// <summary>
+        /// Gets the memory configuration with reduced MaxMemory to work around persistence bug.
+        /// </summary>
+        private static MemoryConfiguration GetMemoryConfig()
+        {
+            // TODO: Remove this method and use default config (IGNITE-5717).
+            return new MemoryConfiguration
+            {
+                MemoryPolicies = new[]
+                {
+                    new MemoryPolicyConfiguration
+                    {
+                        Name = MemoryConfiguration.DefaultDefaultMemoryPolicyName,
+                        InitialSize = 512*1024*1024,
+                        MaxSize = 512*1024*1024
+                    }
+                }
+            };
+        }
     }
 }