You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/07/11 09:03:25 UTC

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

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-2.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;