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 2016/06/20 12:52:41 UTC

[01/50] ignite git commit: IGNITE-3287: IGFS Delete in PRIMARY/DUAL modes goes through the same path.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3341 [created] d4d6b0c54


IGNITE-3287: IGFS Delete in PRIMARY/DUAL modes goes through the same path.


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

Branch: refs/heads/ignite-3341
Commit: 1cb291c68962ac911814d57917ba7a95d7b90c37
Parents: ec5706f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 10 11:05:48 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 10 11:05:48 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDeleteResult.java       |  62 +++++
 .../internal/processors/igfs/IgfsImpl.java      |  30 +--
 .../processors/igfs/IgfsMetaManager.java        | 265 ++++++-------------
 .../internal/processors/igfs/IgfsPathIds.java   |  10 +
 .../processors/igfs/IgfsAbstractSelfTest.java   |  14 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |  20 +-
 6 files changed, 177 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1cb291c6/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java
new file mode 100644
index 0000000..ff42762
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteResult.java
@@ -0,0 +1,62 @@
+/*
+ * 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.igfs;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Result of deletion in IGFS.
+ */
+public class IgfsDeleteResult {
+    /** Success flag. */
+    private final boolean success;
+
+    /** Entry info. */
+    private final IgfsEntryInfo info;
+
+    /**
+     * Constructor.
+     *
+     * @param success Success flag.
+     * @param info Entry info.
+     */
+    public IgfsDeleteResult(boolean success, @Nullable IgfsEntryInfo info) {
+        this.success = success;
+        this.info = info;
+    }
+
+    /**
+     * @return Success flag.
+     */
+    public boolean success() {
+        return success;
+    }
+
+    /**
+     * @return Entry info.
+     */
+    public IgfsEntryInfo info() {
+        return info;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsDeleteResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1cb291c6/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index bc26ace..0808619 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -104,7 +104,6 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_WRITE;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
@@ -723,38 +722,21 @@ public final class IgfsImpl implements IgfsEx {
                 if (IgfsPath.SLASH.equals(path.toString()))
                     return false;
 
-                IgfsMode mode = resolveMode(path);
-
                 Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
 
-                boolean res = false;
-
-                FileDescriptor desc = getFileDescriptor(path);
-
-                if (childrenModes.contains(PRIMARY)) {
-                    if (desc != null) {
-                        IgniteUuid deletedId = meta.softDelete(path, recursive);
-
-                        res = deletedId != null;
-                    }
-                    else if (mode == PRIMARY)
-                        checkConflictWithPrimary(path);
-                }
-
-                if (childrenModes.contains(DUAL_SYNC) || childrenModes.contains(DUAL_ASYNC)) {
-                    assert secondaryFs != null;
+                boolean dual = childrenModes.contains(DUAL_SYNC) ||childrenModes.contains(DUAL_ASYNC);
 
+                if (dual)
                     await(path);
 
-                    res |= meta.deleteDual(secondaryFs, path, recursive);
-                }
+                IgfsDeleteResult res = meta.softDelete(path, recursive, dual ? secondaryFs : null);
 
                 // Record event if needed.
-                if (res && desc != null)
+                if (res.success() && res.info() != null)
                     IgfsUtils.sendEvents(igfsCtx.kernalContext(), path,
-                            desc.isFile ? EVT_IGFS_FILE_DELETED : EVT_IGFS_DIR_DELETED);
+                        res.info().isFile() ? EVT_IGFS_FILE_DELETED : EVT_IGFS_DIR_DELETED);
 
-                return res;
+                return res.success();
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1cb291c6/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 91bb334..2fb6066 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -1150,153 +1150,129 @@ public class IgfsMetaManager extends IgfsManager {
      *
      * @param path Path.
      * @param recursive Recursive flag.
+     * @param secondaryFs Secondary file system (optional).
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
-    IgniteUuid softDelete(final IgfsPath path, final boolean recursive) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                validTxState(false);
-
-                IgfsPathIds pathIds = pathIds(path);
-
-                // Continue only if the whole path present.
-                if (!pathIds.allExists())
-                    return null; // A fragment of the path no longer exists.
-
-                IgniteUuid victimId = pathIds.lastId();
-                String victimName = pathIds.lastPart();
-
-                if (IgfsUtils.isRootId(victimId))
-                    throw new IgfsException("Cannot remove root directory");
-
-                // Prepare IDs to lock.
-                SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
-
-                pathIds.addExistingIds(allIds, relaxed);
-
-                IgniteUuid trashId = IgfsUtils.randomTrashId();
+    IgfsDeleteResult softDelete(final IgfsPath path, final boolean recursive, @Nullable IgfsSecondaryFileSystem secondaryFs)
+        throws IgniteCheckedException {
+        while (true) {
+            if (busyLock.enterBusy()) {
+                try {
+                    validTxState(false);
 
-                allIds.add(trashId);
+                    IgfsPathIds pathIds = pathIds(path);
 
-                try (IgniteInternalTx tx = startTx()) {
-                    // Lock participants.
-                    Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
+                    boolean relaxed0 = relaxed;
 
-                    // Ensure that all participants are still in place.
-                    if (!pathIds.verifyIntegrity(lockInfos, relaxed))
-                        return null;
+                    if (!pathIds.allExists()) {
+                        if (secondaryFs == null)
+                            // Return early if target path doesn't exist and we do not have secondary file system.
+                            return new IgfsDeleteResult(false, null);
+                        else
+                            // If path is missing partially, we need more serious locking for DUAL mode.
+                            relaxed0 = false;
+                    }
 
-                    IgfsEntryInfo victimInfo = lockInfos.get(victimId);
+                    IgniteUuid victimId = pathIds.lastId();
+                    String victimName = pathIds.lastPart();
 
-                    // Cannot delete non-empty directory if recursive flag is not set.
-                    if (!recursive && victimInfo.hasChildren())
-                        throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
-                            "empty and recursive flag is not set).");
+                    if (IgfsUtils.isRootId(victimId))
+                        throw new IgfsException("Cannot remove root directory");
 
-                    // Prepare trash data.
-                    IgfsEntryInfo trashInfo = lockInfos.get(trashId);
-                    final String trashName = victimId.toString();
+                    // Prepare IDs to lock.
+                    SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                    assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
-                        "destination directory (file already exists) [destName=" + trashName + ']';
+                    pathIds.addExistingIds(allIds, relaxed0);
 
-                    IgniteUuid parentId = pathIds.lastParentId();
-                    IgfsEntryInfo parentInfo = lockInfos.get(parentId);
+                    IgniteUuid trashId = IgfsUtils.randomTrashId();
 
-                    transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
+                    allIds.add(trashId);
 
-                    tx.commit();
+                    try (IgniteInternalTx tx = startTx()) {
+                        // Lock participants.
+                        Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
 
-                    signalDeleteWorker();
+                        if (!pathIds.allExists()) {
+                            // We need to ensure that the last locked info is not linked with expected child.
+                            // Otherwise there was some ocncurrent file system update and we have to re-try.
+                            assert secondaryFs != null;
 
-                    return victimId;
-                }
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to perform soft delete because Grid is " +
-                "stopping [path=" + path + ']');
-    }
+                            // Find the last locked index
+                            IgfsEntryInfo lastLockedInfo = null;
+                            int lastLockedIdx = -1;
 
-    /**
-     * Move path to the trash directory in existing transaction.
-     *
-     * @param parentId Parent ID.
-     * @param name Path name.
-     * @param id Path ID.
-     * @param trashId Trash ID.
-     * @return ID of an entry located directly under the trash directory.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("RedundantCast")
-    @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id,
-        IgniteUuid trashId) throws IgniteCheckedException {
-        validTxState(true);
+                            while (lastLockedIdx < pathIds.lastExistingIndex()) {
+                                IgfsEntryInfo nextInfo = lockInfos.get(pathIds.id(lastLockedIdx + 1));
 
-        IgniteUuid resId;
+                                if (nextInfo != null) {
+                                    lastLockedInfo = nextInfo;
+                                    lastLockedIdx++;
+                                }
+                                else
+                                    break;
+                            }
 
-        if (parentId == null) {
-            // Handle special case when we deleting root directory.
-            assert IgfsUtils.ROOT_ID.equals(id);
+                            assert lastLockedIdx < pathIds.count();
 
-            IgfsEntryInfo rootInfo = getInfo(IgfsUtils.ROOT_ID);
+                            if (lastLockedInfo != null) {
+                                String part = pathIds.part(lastLockedIdx + 1);
 
-            if (rootInfo == null)
-                return null; // Root was never created.
+                                if (lastLockedInfo.listing().containsKey(part))
+                                    continue;
+                            }
+                        }
 
-            // Ensure trash directory existence.
-            createSystemDirectoryIfAbsent(trashId);
+                        // Ensure that all participants are still in place.
+                        if (!pathIds.allExists() || !pathIds.verifyIntegrity(lockInfos, relaxed0)) {
+                            // For DUAL mode we will try to update the underlying FS still. Note we do that inside TX.
+                            if (secondaryFs != null) {
+                                boolean res = secondaryFs.delete(path, recursive);
 
-            Map<String, IgfsListingEntry> rootListing = rootInfo.listing();
+                                return new IgfsDeleteResult(res, null);
+                            }
+                            else
+                                return new IgfsDeleteResult(false, null);
+                        }
 
-            if (!rootListing.isEmpty()) {
-                IgniteUuid[] lockIds = new IgniteUuid[rootInfo.listing().size()];
+                        IgfsEntryInfo victimInfo = lockInfos.get(victimId);
 
-                int i = 0;
+                        // Cannot delete non-empty directory if recursive flag is not set.
+                        if (!recursive && victimInfo.hasChildren())
+                            throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
+                                "empty and recursive flag is not set).");
 
-                for (IgfsListingEntry entry : rootInfo.listing().values())
-                    lockIds[i++] = entry.fileId();
+                        // Prepare trash data.
+                        IgfsEntryInfo trashInfo = lockInfos.get(trashId);
+                        final String trashName = victimId.toString();
 
-                // Lock children IDs in correct order.
-                lockIds(lockIds);
+                        assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " +
+                            "destination directory (file already exists) [destName=" + trashName + ']';
 
-                // Construct new info and move locked entries from root to it.
-                Map<String, IgfsListingEntry> transferListing = new HashMap<>();
+                        IgniteUuid parentId = pathIds.lastParentId();
+                        IgfsEntryInfo parentInfo = lockInfos.get(parentId);
 
-                transferListing.putAll(rootListing);
+                        // Propagate call to the secondary file system.
+                        if (secondaryFs != null && !secondaryFs.delete(path, recursive))
+                            return new IgfsDeleteResult(false, null);
 
-                IgfsEntryInfo newInfo = IgfsUtils.createDirectory(
-                    IgniteUuid.randomUuid(),
-                    transferListing,
-                    (Map<String,String>)null
-                );
+                        transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName);
 
-                createNewEntry(newInfo, trashId, newInfo.id().toString());
+                        tx.commit();
 
-                // Remove listing entries from root.
-                for (Map.Entry<String, IgfsListingEntry> entry : transferListing.entrySet())
-                    id2InfoPrj.invoke(IgfsUtils.ROOT_ID,
-                        new IgfsMetaDirectoryListingRemoveProcessor(entry.getKey(), entry.getValue().fileId()));
+                        signalDeleteWorker();
 
-                resId = newInfo.id();
+                        return new IgfsDeleteResult(true, victimInfo);
+                    }
+                }
+                finally {
+                    busyLock.leaveBusy();
+                }
             }
             else
-                resId = null;
+                throw new IllegalStateException("Failed to perform soft delete because Grid is " +
+                    "stopping [path=" + path + ']');
         }
-        else {
-            // Ensure trash directory existence.
-            createSystemDirectoryIfAbsent(trashId);
-
-            moveNonTx(id, name, parentId, id.toString(), trashId);
-
-            resId = id;
-        }
-
-        return resId;
     }
 
     /**
@@ -2273,71 +2249,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Delete path in DUAL mode.
-     *
-     * @param fs Secondary file system.
-     * @param path Path to update.
-     * @param recursive Recursive flag.
-     * @return Operation result.
-     * @throws IgniteCheckedException If delete failed.
-     */
-    public boolean deleteDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final boolean recursive)
-        throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                assert fs != null;
-                assert path != null;
-
-                final IgniteUuid trashId = IgfsUtils.randomTrashId();
-
-                SynchronizationTask<Boolean> task = new SynchronizationTask<Boolean>() {
-                    @Override public Boolean onSuccess(Map<IgfsPath, IgfsEntryInfo> infos) throws Exception {
-                        IgfsEntryInfo info = infos.get(path);
-
-                        if (info == null)
-                            return false; // File doesn't exist in the secondary file system.
-
-                        if (!fs.delete(path, recursive))
-                            return false; // Delete failed remotely.
-
-                        if (path.parent() != null) {
-                            assert infos.containsKey(path.parent());
-
-                            softDeleteNonTx(infos.get(path.parent()).id(), path.name(), info.id(), trashId);
-                        }
-                        else {
-                            assert IgfsUtils.ROOT_ID.equals(info.id());
-
-                            softDeleteNonTx(null, path.name(), info.id(), trashId);
-                        }
-
-                        return true; // No additional handling is required.
-                    }
-
-                    @Override public Boolean onFailure(@Nullable Exception err) throws IgniteCheckedException {
-                        U.error(log, "Path delete in DUAL mode failed [path=" + path + ", recursive=" + recursive + ']',
-                            err);
-
-                        throw new IgniteCheckedException("Failed to delete the path due to secondary file system " +
-                            "exception: ", err);
-                    }
-                };
-
-                Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(trashId), path);
-
-                signalDeleteWorker();
-
-                return res;
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to delete in DUAL mode because Grid is stopping: " + path);
-    }
-
-    /**
      * Update path in DUAL mode.
      *
      * @param fs Secondary file system.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1cb291c6/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
index 446495e..b710ba2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java
@@ -256,6 +256,16 @@ public class IgfsPathIds {
     }
 
     /**
+     * Get ID at the give index.
+     *
+     * @param idx Index.
+     * @return ID.
+     */
+    public IgniteUuid id(int idx) {
+        return idx <= lastExistingIdx ? ids[idx] : surrogateId(idx);
+    }
+
+    /**
      * Get surrogate ID at the given index.
      *
      * @param idx Index.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1cb291c6/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 261a494..ad8c9e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -2899,11 +2899,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException {
-        for (IgfsPath path : paths) {
-            assert igfs.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + igfs.name() +
-                ", path=" + path + ']';
+        for (IgfsPath path : paths)
             assert igfs.exists(path) : "Path doesn't exist [igfs=" + igfs.name() + ", path=" + path + ']';
-        }
     }
 
     /**
@@ -2952,11 +2949,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception {
-        for (IgfsPath path : paths) {
-            assert igfs.context().meta().fileId(path) == null : "Path exists [igfs=" + igfs.name() + ", path=" +
-                path + ']';
+        for (IgfsPath path : paths)
             assert !igfs.exists(path) : "Path exists [igfs=" + igfs.name() + ", path=" + path + ']';
-        }
     }
 
     /**
@@ -2968,10 +2962,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected void checkNotExist(UniversalFileSystemAdapter uni, IgfsPath... paths) throws Exception {
         IgfsEx ex = uni.unwrap(IgfsEx.class);
+
         for (IgfsPath path : paths) {
             if (ex != null)
-                assert ex.context().meta().fileId(path) == null : "Path exists [igfs=" + ex.name() + ", path=" +
-                    path + ']';
+                assert !ex.exists(path) : "Path exists [igfs=" + ex.name() + ", path=" + path + ']';
 
             assert !uni.exists(path.toString()) : "Path exists [igfs=" + uni.name() + ", path=" + path + ']';
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1cb291c6/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index 039bf8d..8b88157 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -191,8 +191,8 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
             assertNull("Unexpected stored properties: " + info, info.properties().get(key2));
         }
 
-        mgr.softDelete(new IgfsPath("/dir"), true);
-        mgr.softDelete(new IgfsPath("/file"), false);
+        mgr.softDelete(new IgfsPath("/dir"), true, null);
+        mgr.softDelete(new IgfsPath("/file"), false, null);
 
         assertNull(mgr.updateProperties(dir.id(), F.asMap("p", "7")));
     }
@@ -328,9 +328,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         mgr.move(path("/a2"), path("/a"));
 
-        IgniteUuid del = mgr.softDelete(path("/a/b/f3"), false);
-
-        assertEquals(f3.id(), del);
+        mgr.softDelete(path("/a/b/f3"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
                 mgr.directoryListing(ROOT_ID));
@@ -341,8 +339,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(b.id()));
 
         //assertEquals(b, mgr.removeIfEmpty(a.id(), "b", b.id(), new IgfsPath("/a/b"), true));
-        del = mgr.softDelete(path("/a/b"), false);
-        assertEquals(b.id(), del);
+        mgr.softDelete(path("/a/b"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -351,8 +348,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
 
         assertEmpty(mgr.directoryListing(b.id()));
 
-        del = mgr.softDelete(path("/a/f2"), false);
-        assertEquals(f2.id(), del);
+        mgr.softDelete(path("/a/f2"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a), "f1", new IgfsListingEntry(f1)),
             mgr.directoryListing(ROOT_ID));
@@ -360,16 +356,14 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(a.id()));
         assertEmpty(mgr.directoryListing(b.id()));
 
-        del = mgr.softDelete(path("/f1"), false);
-        assertEquals(f1.id(), del);
+        mgr.softDelete(path("/f1"), false, null);
 
         assertEquals(F.asMap("a", new IgfsListingEntry(a)), mgr.directoryListing(ROOT_ID));
 
         assertEmpty(mgr.directoryListing(a.id()));
         assertEmpty(mgr.directoryListing(b.id()));
 
-        del = mgr.softDelete(path("/a"), false);
-        assertEquals(a.id(), del);
+        mgr.softDelete(path("/a"), false, null);
 
         assertEmpty(mgr.directoryListing(ROOT_ID));
         assertEmpty(mgr.directoryListing(a.id()));


[09/50] ignite git commit: IGNITE-2938: IGFS: Puts during secondary file reads are now performed synchronously and with proper semantics.

Posted by vo...@apache.org.
IGNITE-2938: IGFS: Puts during secondary file reads are now performed synchronously and with proper semantics.


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

Branch: refs/heads/ignite-3341
Commit: 98a0990c90fbfe5737f4f0f2d1c34a84fd0a6dde
Parents: 54425bf
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Jun 14 17:39:44 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Jun 14 17:39:44 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  |  18 +-
 .../processors/igfs/IgfsDataManager.java        | 166 ++++---------------
 .../igfs/data/IgfsDataPutProcessor.java         |  99 +++++++++++
 3 files changed, 147 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/98a0990c/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
index 074636a..88b0c28 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/FileSystemConfiguration.java
@@ -704,7 +704,9 @@ public class FileSystemConfiguration {
      * Gets maximum timeout awaiting for trash purging in case data cache oversize is detected.
      *
      * @return Maximum timeout awaiting for trash purging in case data cache oversize is detected.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public long getTrashPurgeTimeout() {
         return trashPurgeTimeout;
     }
@@ -713,7 +715,9 @@ public class FileSystemConfiguration {
      * Sets maximum timeout awaiting for trash purging in case data cache oversize is detected.
      *
      * @param trashPurgeTimeout Maximum timeout awaiting for trash purging in case data cache oversize is detected.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public void setTrashPurgeTimeout(long trashPurgeTimeout) {
         this.trashPurgeTimeout = trashPurgeTimeout;
     }
@@ -724,8 +728,10 @@ public class FileSystemConfiguration {
      * In case no executor service is provided, default one will be created with maximum amount of threads equals
      * to amount of processor cores.
      *
-     * @return Get DUAL mode put operation executor service
+     * @return Get DUAL mode put operation executor service.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     @Nullable public ExecutorService getDualModePutExecutorService() {
         return dualModePutExec;
     }
@@ -734,7 +740,9 @@ public class FileSystemConfiguration {
      * Set DUAL mode put operations executor service.
      *
      * @param dualModePutExec Dual mode put operations executor service.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public void setDualModePutExecutorService(ExecutorService dualModePutExec) {
         this.dualModePutExec = dualModePutExec;
     }
@@ -743,7 +751,9 @@ public class FileSystemConfiguration {
      * Get DUAL mode put operation executor service shutdown flag.
      *
      * @return DUAL mode put operation executor service shutdown flag.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public boolean getDualModePutExecutorServiceShutdown() {
         return dualModePutExecShutdown;
     }
@@ -752,7 +762,9 @@ public class FileSystemConfiguration {
      * Set DUAL mode put operations executor service shutdown flag.
      *
      * @param dualModePutExecShutdown Dual mode put operations executor service shutdown flag.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public void setDualModePutExecutorServiceShutdown(boolean dualModePutExecShutdown) {
         this.dualModePutExecShutdown = dualModePutExecShutdown;
     }
@@ -766,7 +778,9 @@ public class FileSystemConfiguration {
      * avoid issues with increasing GC pauses or out-of-memory error.
      *
      * @return Maximum amount of pending data read from the secondary file system
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public long getDualModeMaxPendingPutsSize() {
         return dualModeMaxPendingPutsSize;
     }
@@ -775,7 +789,9 @@ public class FileSystemConfiguration {
      * Set maximum amount of data in pending put operations.
      *
      * @param dualModeMaxPendingPutsSize Maximum amount of data in pending put operations.
+     * @deprecated Not used any more.
      */
+    @Deprecated
     public void setDualModeMaxPendingPutsSize(long dualModeMaxPendingPutsSize) {
         this.dualModeMaxPendingPutsSize = dualModeMaxPendingPutsSize;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/98a0990c/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index 34d77f9..cb2b630 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters;
+import org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -53,7 +54,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -78,10 +78,8 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
@@ -98,9 +96,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  * Cache based file's data container.
  */
 public class IgfsDataManager extends IgfsManager {
-    /** IGFS. */
-    private IgfsEx igfs;
-
     /** Data internal cache. */
     private IgniteInternalCache<IgfsBlockKey, byte[]> dataCachePrj;
 
@@ -143,31 +138,10 @@ public class IgfsDataManager extends IgfsManager {
     /** Async file delete worker. */
     private AsyncDeleteWorker delWorker;
 
-    /** Trash purge timeout. */
-    private long trashPurgeTimeout;
-
     /** On-going remote reads futures. */
     private final ConcurrentHashMap8<IgfsBlockKey, IgniteInternalFuture<byte[]>> rmtReadFuts =
         new ConcurrentHashMap8<>();
 
-    /** Executor service for puts in dual mode */
-    private volatile ExecutorService putExecSvc;
-
-    /** Executor service for puts in dual mode shutdown flag. */
-    private volatile boolean putExecSvcShutdown;
-
-    /** Maximum amount of data in pending puts. */
-    private volatile long maxPendingPuts;
-
-    /** Current amount of data in pending puts. */
-    private long curPendingPuts;
-
-    /** Lock for pending puts. */
-    private final Lock pendingPutsLock = new ReentrantLock();
-
-    /** Condition for pending puts. */
-    private final Condition pendingPutsCond = pendingPutsLock.newCondition();
-
     /**
      *
      */
@@ -182,8 +156,6 @@ public class IgfsDataManager extends IgfsManager {
 
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
-        igfs = igfsCtx.igfs();
-
         dataCacheStartLatch = new CountDownLatch(1);
 
         String igfsName = igfsCtx.configuration().getName();
@@ -216,23 +188,6 @@ public class IgfsDataManager extends IgfsManager {
 
         igfsSvc = igfsCtx.kernalContext().getIgfsExecutorService();
 
-        trashPurgeTimeout = igfsCtx.configuration().getTrashPurgeTimeout();
-
-        putExecSvc = igfsCtx.configuration().getDualModePutExecutorService();
-
-        if (putExecSvc != null)
-            putExecSvcShutdown = igfsCtx.configuration().getDualModePutExecutorServiceShutdown();
-        else {
-            int coresCnt = Runtime.getRuntime().availableProcessors();
-
-            // Note that we do not pre-start threads here as IGFS pool may not be needed.
-            putExecSvc = new IgniteThreadPoolExecutor(coresCnt, coresCnt, 0, new LinkedBlockingDeque<Runnable>());
-
-            putExecSvcShutdown = true;
-        }
-
-        maxPendingPuts = igfsCtx.configuration().getDualModeMaxPendingPutsSize();
-
         delWorker = new AsyncDeleteWorker(igfsCtx.kernalContext().gridName(),
             "igfs-" + igfsName + "-delete-worker", log);
     }
@@ -282,9 +237,6 @@ public class IgfsDataManager extends IgfsManager {
         catch (IgniteInterruptedCheckedException e) {
             log.warning("Got interrupter while waiting for delete worker to stop (will continue stopping).", e);
         }
-
-        if (putExecSvcShutdown)
-            U.shutdownNow(getClass(), putExecSvc, log);
     }
 
     /**
@@ -308,6 +260,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param prevAffKey Affinity key of previous block.
      * @return Affinity key.
      */
+    @SuppressWarnings("ConstantConditions")
     public IgniteUuid nextAffinityKey(@Nullable IgniteUuid prevAffKey) {
         // Do not generate affinity key for non-affinity nodes.
         if (!dataCache.context().affinityNode())
@@ -371,8 +324,6 @@ public class IgfsDataManager extends IgfsManager {
     @Nullable public IgniteInternalFuture<byte[]> dataBlock(final IgfsEntryInfo fileInfo, final IgfsPath path,
         final long blockIdx, @Nullable final IgfsSecondaryFileSystemPositionedReadable secReader)
         throws IgniteCheckedException {
-        //assert validTxState(any); // Allow this method call for any transaction state.
-
         assert fileInfo != null;
         assert blockIdx >= 0;
 
@@ -435,7 +386,7 @@ public class IgfsDataManager extends IgfsManager {
 
                                 rmtReadFut.onDone(res);
 
-                                putSafe(key, res);
+                                putBlock(fileInfo.blockSize(), key, res);
 
                                 metrics.addReadBlocks(1, 1);
                             }
@@ -471,6 +422,26 @@ public class IgfsDataManager extends IgfsManager {
     }
 
     /**
+     * Stores the given block in data cache.
+     *
+     * @param blockSize The size of the block.
+     * @param key The data cache key of the block.
+     * @param data The new value of the block.
+     */
+    private void putBlock(int blockSize, IgfsBlockKey key, byte[] data) throws IgniteCheckedException {
+        if (data.length < blockSize)
+            // partial (incomplete) block:
+            dataCachePrj.invoke(key, new IgfsDataPutProcessor(data));
+        else {
+            // whole block:
+            assert data.length == blockSize;
+
+            dataCachePrj.put(key, data);
+        }
+    }
+
+
+    /**
      * Registers write future in igfs data manager.
      *
      * @param fileId File ID.
@@ -680,7 +651,7 @@ public class IgfsDataManager extends IgfsManager {
                                 byte[] val = vals.get(colocatedKey);
 
                                 if (val != null) {
-                                    dataCachePrj.put(key, val);
+                                    putBlock(fileInfo.blockSize(), key, val);
 
                                     tx.commit();
                                 }
@@ -744,7 +715,6 @@ public class IgfsDataManager extends IgfsManager {
      */
     public Collection<IgfsBlockLocation> affinity(IgfsEntryInfo info, long start, long len, long maxLen)
         throws IgniteCheckedException {
-        assert validTxState(false);
         assert info.isFile() : "Failed to get affinity (not a file): " + info;
         assert start >= 0 : "Start position should not be negative: " + start;
         assert len >= 0 : "Part length should not be negative: " + len;
@@ -974,21 +944,6 @@ public class IgfsDataManager extends IgfsManager {
     }
 
     /**
-     * Check transaction is (not) started.
-     *
-     * @param inTx Expected transaction state.
-     * @return Transaction state is correct.
-     */
-    private boolean validTxState(boolean inTx) {
-        boolean txState = inTx == (dataCachePrj.tx() != null);
-
-        assert txState : (inTx ? "Method cannot be called outside transaction: " :
-            "Method cannot be called in transaction: ") + dataCachePrj.tx();
-
-        return txState;
-    }
-
-    /**
      * @param fileId File ID.
      * @param node Node to process blocks on.
      * @param blocks Blocks to put in cache.
@@ -1056,10 +1011,11 @@ public class IgfsDataManager extends IgfsManager {
      * @param colocatedKey Block key.
      * @param startOff Data start offset within block.
      * @param data Data to write.
+     * @param blockSize The block size.
      * @throws IgniteCheckedException If update failed.
      */
     private void processPartialBlockWrite(IgniteUuid fileId, IgfsBlockKey colocatedKey, int startOff,
-        byte[] data) throws IgniteCheckedException {
+        byte[] data, int blockSize) throws IgniteCheckedException {
         if (dataCachePrj.igfsDataSpaceUsed() >= dataCachePrj.igfsDataSpaceMax()) {
             final WriteCompletionFuture completionFut = pendingWrites.get(fileId);
 
@@ -1090,7 +1046,7 @@ public class IgfsDataManager extends IgfsManager {
 
         // If writing from block beginning, just put and return.
         if (startOff == 0) {
-            dataCachePrj.put(colocatedKey, data);
+            putBlock(blockSize, colocatedKey, data);
 
             return;
         }
@@ -1151,67 +1107,6 @@ public class IgfsDataManager extends IgfsManager {
     }
 
     /**
-     * Put data block read from the secondary file system to the cache.
-     *
-     * @param key Key.
-     * @param data Data.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void putSafe(final IgfsBlockKey key, final byte[] data) throws IgniteCheckedException {
-        assert key != null;
-        assert data != null;
-
-        if (maxPendingPuts > 0) {
-            pendingPutsLock.lock();
-
-            try {
-                while (curPendingPuts > maxPendingPuts)
-                    pendingPutsCond.await(2000, TimeUnit.MILLISECONDS);
-
-                curPendingPuts += data.length;
-            }
-            catch (InterruptedException ignore) {
-                throw new IgniteCheckedException("Failed to put IGFS data block into cache due to interruption: " + key);
-            }
-            finally {
-                pendingPutsLock.unlock();
-            }
-        }
-
-        Runnable task = new Runnable() {
-            @Override public void run() {
-                try {
-                    dataCachePrj.put(key, data);
-                }
-                catch (IgniteCheckedException e) {
-                    U.warn(log, "Failed to put IGFS data block into cache [key=" + key + ", err=" + e + ']');
-                }
-                finally {
-                    if (maxPendingPuts > 0) {
-                        pendingPutsLock.lock();
-
-                        try {
-                            curPendingPuts -= data.length;
-
-                            pendingPutsCond.signalAll();
-                        }
-                        finally {
-                            pendingPutsLock.unlock();
-                        }
-                    }
-                }
-            }
-        };
-
-        try {
-            putExecSvc.submit(task);
-        }
-        catch (RejectedExecutionException ignore) {
-            task.run();
-        }
-    }
-
-    /**
      * @param blocks Blocks to write.
      * @return Future that will be completed after put is done.
      */
@@ -1261,6 +1156,7 @@ public class IgfsDataManager extends IgfsManager {
      * @param nodeId Node ID.
      * @param ackMsg Write acknowledgement message.
      */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     private void processAckMessage(UUID nodeId, IgfsAckMessage ackMsg) {
         try {
             ackMsg.finishUnmarshal(igfsCtx.kernalContext().config().getMarshaller(), null);
@@ -1343,6 +1239,7 @@ public class IgfsDataManager extends IgfsManager {
          * @throws IgniteCheckedException If failed.
          * @return Data remainder if {@code flush} flag is {@code false}.
          */
+        @SuppressWarnings("ConstantConditions")
         @Nullable public byte[] storeDataBlocks(
             IgfsEntryInfo fileInfo,
             long reservedLen,
@@ -1456,7 +1353,7 @@ public class IgfsDataManager extends IgfsManager {
 
                 if (size != blockSize) {
                     // Partial writes must be always synchronous.
-                    processPartialBlockWrite(id, key, block == first ? off : 0, portion);
+                    processPartialBlockWrite(id, key, block == first ? off : 0, portion, blockSize);
 
                     writtenTotal++;
                 }
@@ -1617,8 +1514,6 @@ public class IgfsDataManager extends IgfsManager {
         protected AsyncDeleteWorker(@Nullable String gridName, String name, IgniteLogger log) {
             super(gridName, name, log);
 
-            long time = System.currentTimeMillis();
-
             stopInfo = IgfsUtils.createDirectory(IgniteUuid.randomUuid());
         }
 
@@ -1642,6 +1537,7 @@ public class IgfsDataManager extends IgfsManager {
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("ConstantConditions")
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             try {
                 while (!isCancelled()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/98a0990c/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/data/IgfsDataPutProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/data/IgfsDataPutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/data/IgfsDataPutProcessor.java
new file mode 100644
index 0000000..2029d4e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/data/IgfsDataPutProcessor.java
@@ -0,0 +1,99 @@
+/*
+ * 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.igfs.data;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.processors.igfs.IgfsBlockKey;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+
+/**
+ * Entry processor to set or replace block byte value.
+ */
+public class IgfsDataPutProcessor implements EntryProcessor<IgfsBlockKey, byte[], Void>, Externalizable, Binarylizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** The new value. */
+    private byte[] newVal;
+
+    /**
+     * Non-arg constructor required by externalizable.
+     */
+    public IgfsDataPutProcessor() {
+        // no-op
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param newVal The new value.
+     */
+    public IgfsDataPutProcessor(byte[] newVal) {
+        assert newVal != null;
+
+        this.newVal = newVal;
+    }
+
+    /** {@inheritDoc} */
+    public Void process(MutableEntry<IgfsBlockKey, byte[]> entry, Object... args)
+        throws EntryProcessorException {
+        byte[] curVal = entry.getValue();
+
+        if (curVal == null || newVal.length > curVal.length)
+            entry.setValue(newVal);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        newVal = U.readByteArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeByteArray(out, newVal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        newVal = reader.rawReader().readByteArray();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        writer.rawWriter().writeByteArray(newVal);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsDataPutProcessor.class, this);
+    }
+}


[27/50] ignite git commit: IGNITE-3238 Javadoc Warning due to cassandra libs usage

Posted by vo...@apache.org.
IGNITE-3238 Javadoc Warning due to cassandra libs usage


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

Branch: refs/heads/ignite-3341
Commit: 95d57506abd172da77e0c8378a8b49f44a174cb2
Parents: 83c26a9
Author: Igor <ir...@gmail.com>
Authored: Fri Jun 10 02:38:41 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Jun 15 15:23:10 2016 +0300

----------------------------------------------------------------------
 modules/cassandra/pom.xml                       |   2 +-
 .../cassandra/bean/CassandraLifeCycleBean.java  | 149 -------------------
 .../ignite/tests/utils/CassandraHelper.java     |   2 +-
 .../tests/utils/CassandraLifeCycleBean.java     | 149 +++++++++++++++++++
 4 files changed, 151 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/95d57506/modules/cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/pom.xml b/modules/cassandra/pom.xml
index d2a7f25..4a1469a 100644
--- a/modules/cassandra/pom.xml
+++ b/modules/cassandra/pom.xml
@@ -165,7 +165,7 @@
             <groupId>org.apache.cassandra</groupId>
             <artifactId>cassandra-all</artifactId>
             <version>${cassandra-all.version}</version>
-            <scope>provided</scope>
+            <scope>test</scope>
             <exclusions>
                 <exclusion>
                     <artifactId>log4j-over-slf4j</artifactId>

http://git-wip-us.apache.org/repos/asf/ignite/blob/95d57506/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java
deleted file mode 100644
index 11f8145..0000000
--- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.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.cache.store.cassandra.bean;
-
-import org.apache.cassandra.service.CassandraDaemon;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.lifecycle.LifecycleBean;
-import org.apache.ignite.lifecycle.LifecycleEventType;
-import org.apache.ignite.resources.LoggerResource;
-
-/**
- * Implementation of {@link LifecycleBean} to start embedded Cassandra instance on Ignite cluster startup
- */
-public class CassandraLifeCycleBean implements LifecycleBean {
-    /** System property specifying Cassandra jmx port */
-    private static final String CASSANDRA_JMX_PORT_PROP = "cassandra.jmx.local.port";
-
-    /** System property specifying Cassandra YAML config file */
-    private static final String CASSANDRA_CONFIG_PROP = "cassandra.config";
-
-    /** Prefix for file path syntax */
-    private static final String FILE_PREFIX = "file:///";
-
-    /** Auto-injected logger instance. */
-    @LoggerResource
-    private IgniteLogger log;
-
-    /** Instance of embedded Cassandra database */
-    private CassandraDaemon embeddedCassandraDaemon;
-
-    /** JMX port for embedded Cassandra instance */
-    private String jmxPort;
-
-    /** YAML config file for embedded Cassandra */
-    private String cassandraCfgFile;
-
-    /**
-     * Returns JMX port for embedded Cassandra
-     * @return JMX port
-     */
-    public String getJmxPort() {
-        return jmxPort;
-    }
-
-    /**
-     * Setter for embedded Cassandra JMX port
-     * @param jmxPort embedded Cassandra JMX port
-     */
-    public void setJmxPort(String jmxPort) {
-        this.jmxPort = jmxPort;
-    }
-
-    /**
-     * Returns embedded Cassandra YAML config file
-     * @return YAML config file
-     */
-    public String getCassandraConfigFile() {
-        return cassandraCfgFile;
-    }
-
-    /**
-     * Setter for embedded Cassandra YAML config file
-     * @param cassandraCfgFile YAML config file
-     */
-    public void setCassandraConfigFile(String cassandraCfgFile) {
-        this.cassandraCfgFile = cassandraCfgFile;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onLifecycleEvent(LifecycleEventType evt) {
-        if (evt == LifecycleEventType.BEFORE_NODE_START)
-            startEmbeddedCassandra();
-        else if (evt == LifecycleEventType.BEFORE_NODE_STOP)
-            stopEmbeddedCassandra();
-    }
-
-    /**
-     * Starts embedded Cassandra instance
-     */
-    private void startEmbeddedCassandra() {
-        if (log != null) {
-            log.info("-------------------------------");
-            log.info("| Starting embedded Cassandra |");
-            log.info("-------------------------------");
-        }
-
-        try {
-            if (jmxPort != null)
-                System.setProperty(CASSANDRA_JMX_PORT_PROP, jmxPort);
-
-            if (cassandraCfgFile != null)
-                System.setProperty(CASSANDRA_CONFIG_PROP, FILE_PREFIX + cassandraCfgFile);
-
-            embeddedCassandraDaemon = new CassandraDaemon(true);
-            embeddedCassandraDaemon.init(null);
-            embeddedCassandraDaemon.start();
-        }
-        catch (Exception e) {
-            throw new RuntimeException("Failed to start embedded Cassandra", e);
-        }
-
-        if (log != null) {
-            log.info("------------------------------");
-            log.info("| Embedded Cassandra started |");
-            log.info("------------------------------");
-        }
-    }
-
-    /**
-     * Stops embedded Cassandra instance
-     */
-    private void stopEmbeddedCassandra() {
-        if (log != null) {
-            log.info("-------------------------------");
-            log.info("| Stopping embedded Cassandra |");
-            log.info("-------------------------------");
-        }
-
-        if (embeddedCassandraDaemon != null) {
-            try {
-                embeddedCassandraDaemon.deactivate();
-            }
-            catch (Throwable e) {
-                throw new RuntimeException("Failed to stop embedded Cassandra", e);
-            }
-        }
-
-        if (log != null) {
-            log.info("------------------------------");
-            log.info("| Embedded Cassandra stopped |");
-            log.info("------------------------------");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/95d57506/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
index d1ef1bd..970be84 100644
--- a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
@@ -29,7 +29,7 @@ import java.net.URL;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.ResourceBundle;
-import org.apache.ignite.cache.store.cassandra.bean.CassandraLifeCycleBean;
+
 import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
 import org.apache.ignite.cache.store.cassandra.session.pool.SessionPool;
 import org.apache.ignite.internal.util.typedef.internal.U;

http://git-wip-us.apache.org/repos/asf/ignite/blob/95d57506/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraLifeCycleBean.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraLifeCycleBean.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraLifeCycleBean.java
new file mode 100644
index 0000000..d70d71e
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraLifeCycleBean.java
@@ -0,0 +1,149 @@
+/*
+ * 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.tests.utils;
+
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.lifecycle.LifecycleBean;
+import org.apache.ignite.lifecycle.LifecycleEventType;
+import org.apache.ignite.resources.LoggerResource;
+
+/**
+ * Implementation of {@link LifecycleBean} to start embedded Cassandra instance on Ignite cluster startup
+ */
+public class CassandraLifeCycleBean implements LifecycleBean {
+    /** System property specifying Cassandra jmx port */
+    private static final String CASSANDRA_JMX_PORT_PROP = "cassandra.jmx.local.port";
+
+    /** System property specifying Cassandra YAML config file */
+    private static final String CASSANDRA_CONFIG_PROP = "cassandra.config";
+
+    /** Prefix for file path syntax */
+    private static final String FILE_PREFIX = "file:///";
+
+    /** Auto-injected logger instance. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Instance of embedded Cassandra database */
+    private CassandraDaemon embeddedCassandraDaemon;
+
+    /** JMX port for embedded Cassandra instance */
+    private String jmxPort;
+
+    /** YAML config file for embedded Cassandra */
+    private String cassandraCfgFile;
+
+    /**
+     * Returns JMX port for embedded Cassandra
+     * @return JMX port
+     */
+    public String getJmxPort() {
+        return jmxPort;
+    }
+
+    /**
+     * Setter for embedded Cassandra JMX port
+     * @param jmxPort embedded Cassandra JMX port
+     */
+    public void setJmxPort(String jmxPort) {
+        this.jmxPort = jmxPort;
+    }
+
+    /**
+     * Returns embedded Cassandra YAML config file
+     * @return YAML config file
+     */
+    public String getCassandraConfigFile() {
+        return cassandraCfgFile;
+    }
+
+    /**
+     * Setter for embedded Cassandra YAML config file
+     * @param cassandraCfgFile YAML config file
+     */
+    public void setCassandraConfigFile(String cassandraCfgFile) {
+        this.cassandraCfgFile = cassandraCfgFile;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLifecycleEvent(LifecycleEventType evt) {
+        if (evt == LifecycleEventType.BEFORE_NODE_START)
+            startEmbeddedCassandra();
+        else if (evt == LifecycleEventType.BEFORE_NODE_STOP)
+            stopEmbeddedCassandra();
+    }
+
+    /**
+     * Starts embedded Cassandra instance
+     */
+    private void startEmbeddedCassandra() {
+        if (log != null) {
+            log.info("-------------------------------");
+            log.info("| Starting embedded Cassandra |");
+            log.info("-------------------------------");
+        }
+
+        try {
+            if (jmxPort != null)
+                System.setProperty(CASSANDRA_JMX_PORT_PROP, jmxPort);
+
+            if (cassandraCfgFile != null)
+                System.setProperty(CASSANDRA_CONFIG_PROP, FILE_PREFIX + cassandraCfgFile);
+
+            embeddedCassandraDaemon = new CassandraDaemon(true);
+            embeddedCassandraDaemon.init(null);
+            embeddedCassandraDaemon.start();
+        }
+        catch (Exception e) {
+            throw new RuntimeException("Failed to start embedded Cassandra", e);
+        }
+
+        if (log != null) {
+            log.info("------------------------------");
+            log.info("| Embedded Cassandra started |");
+            log.info("------------------------------");
+        }
+    }
+
+    /**
+     * Stops embedded Cassandra instance
+     */
+    private void stopEmbeddedCassandra() {
+        if (log != null) {
+            log.info("-------------------------------");
+            log.info("| Stopping embedded Cassandra |");
+            log.info("-------------------------------");
+        }
+
+        if (embeddedCassandraDaemon != null) {
+            try {
+                embeddedCassandraDaemon.deactivate();
+            }
+            catch (Throwable e) {
+                throw new RuntimeException("Failed to stop embedded Cassandra", e);
+            }
+        }
+
+        if (log != null) {
+            log.info("------------------------------");
+            log.info("| Embedded Cassandra stopped |");
+            log.info("------------------------------");
+        }
+    }
+}


[37/50] ignite git commit: GridServiceProcessor: guard submit in 'depExe' pool to avoid RejectedExecutionException.

Posted by vo...@apache.org.
GridServiceProcessor: guard submit in 'depExe' pool to avoid RejectedExecutionException.


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

Branch: refs/heads/ignite-3341
Commit: f260589d79833b86271f033a37404c542933bc6e
Parents: 7da0a02
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 16 12:09:15 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 16 12:09:15 2016 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/f260589d/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 853e6bf..add90e2 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
@@ -35,7 +35,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.cache.Cache;
 import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
 import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -60,7 +59,6 @@ import org.apache.ignite.internal.processors.cache.CacheIteratorConverter;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
@@ -1312,11 +1310,19 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     private class ServiceEntriesListener implements CacheEntryUpdatedListener<Object, Object> {
         /** {@inheritDoc} */
         @Override public void onUpdated(final Iterable<CacheEntryEvent<?, ?>> deps) {
-            depExe.submit(new BusyRunnable() {
-                @Override public void run0() {
-                    onSystemCacheUpdated(deps);
-                }
-            });
+            if (!busyLock.enterBusy())
+                return;
+
+            try {
+                depExe.submit(new BusyRunnable() {
+                    @Override public void run0() {
+                        onSystemCacheUpdated(deps);
+                    }
+                });
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
         }
     }
 


[28/50] ignite git commit: ignite-3209 Review fixes

Posted by vo...@apache.org.
ignite-3209 Review fixes


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

Branch: refs/heads/ignite-3341
Commit: 2a3ffc15d661cb9f097a65e2760afb3bcbad957b
Parents: ebe5658
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 15 15:44:58 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jun 15 15:44:58 2016 +0300

----------------------------------------------------------------------
 .../processors/task/GridTaskWorker.java         | 154 ++++++++++---------
 1 file changed, 80 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2a3ffc15/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 0c522ad..fc56893 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -69,7 +69,6 @@ import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.closure.AffinityTask;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
@@ -79,7 +78,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException;
-import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
@@ -683,6 +681,12 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
             // job response was changed in this method apply.
             boolean selfOccupied = false;
 
+            IgniteInternalFuture<?> affFut = null;
+
+            boolean waitForAffTop = false;
+
+            final GridJobExecuteResponse failoverRes = res;
+
             try {
                 synchronized (mux) {
                     // If task is not waiting for responses,
@@ -852,25 +856,19 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                         }
 
                         case FAILOVER: {
-                            IgniteInternalFuture<Boolean> fut = failover(res, jobRes, getTaskTopology());
+                            if (affKey != null) {
+                                AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
 
-                            final GridJobResultImpl jobRes0 = jobRes;
-
-                            fut.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
-                                @Override public void apply(IgniteInternalFuture<Boolean> fut0) {
-                                    try {
-                                        Boolean res = fut0.get();
+                                affFut = ctx.cache().context().exchange().affinityReadyFuture(topVer);
+                            }
 
-                                        if (res)
-                                            sendFailoverRequest(jobRes0);
-                                    }
-                                    catch (IgniteCheckedException e) {
-                                        U.error(log, "Failed to failover task [ses=" + ses + ", err=" + e + ']', e);
+                            if (affFut != null && !affFut.isDone()) {
+                                waitForAffTop = true;
 
-                                        finishTask(null, e);
-                                    }
-                                }
-                            });
+                                jobRes.resetResponse();
+                            }
+                            else if (!failover(res, jobRes, getTaskTopology()))
+                                plc = null;
 
                             break;
                         }
@@ -878,17 +876,24 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                 }
 
                 // Outside of synchronization.
-                if (plc != null && plc != FAILOVER) {
-                    evtLsnr.onJobFinished(this, jobRes.getSibling());
+                if (plc != null && !waitForAffTop) {
+                    // Handle failover.
+                    if (plc == FAILOVER)
+                        sendFailoverRequest(jobRes);
+                    else {
+                        evtLsnr.onJobFinished(this, jobRes.getSibling());
 
-                    if (plc == ComputeJobResultPolicy.REDUCE)
-                        reduce(results);
+                        if (plc == ComputeJobResultPolicy.REDUCE)
+                            reduce(results);
+                    }
                 }
             }
             catch (IgniteCheckedException e) {
                 U.error(log, "Failed to obtain topology [ses=" + ses + ", err=" + e + ']', e);
 
                 finishTask(null, e);
+
+                waitForAffTop = false;
             }
             finally {
                 // Open up job for processing responses.
@@ -907,6 +912,18 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                     res = delayedRess.poll();
                 }
             }
+
+            if (waitForAffTop && affFut != null) {
+                affFut.listen(new IgniteInClosure<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> fut0) {
+                        ctx.closure().runLocalSafe(new Runnable() {
+                            @Override public void run() {
+                                onResponse(failoverRes);
+                            }
+                        }, false);
+                    }
+                });
+            }
         }
     }
 
@@ -1056,74 +1073,63 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
      * @param top Topology.
      * @return {@code True} if fail-over SPI returned a new node.
      */
-    private IgniteInternalFuture<Boolean> failover(
-        final GridJobExecuteResponse res,
-        final GridJobResultImpl jobRes,
-        final Collection<? extends ClusterNode> top
+    private boolean failover(
+        GridJobExecuteResponse res,
+        GridJobResultImpl jobRes,
+        Collection<? extends ClusterNode> top
     ) {
-        IgniteInternalFuture<?> affFut = null;
-
-        if (affKey != null) {
-            AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
-
-            affFut = ctx.cache().context().exchange().affinityReadyFuture(topVer);
-        }
-
-        if (affFut == null)
-            affFut = new GridFinishedFuture();
+        assert Thread.holdsLock(mux);
 
-        return affFut.chain(new IgniteClosure<IgniteInternalFuture<?>, Boolean>() {
-            @Override public Boolean apply(IgniteInternalFuture<?> fut0) {
-                synchronized (mux) {
-                    try {
-                        ctx.resource().invokeAnnotated(dep, jobRes.getJob(), ComputeJobBeforeFailover.class);
+        try {
+            ctx.resource().invokeAnnotated(dep, jobRes.getJob(), ComputeJobBeforeFailover.class);
 
-                        // Map to a new node.
-                        ClusterNode node = ctx.failover().failover(ses, jobRes, new ArrayList<>(top), affKey, affCache);
+            // Map to a new node.
+            ClusterNode node = ctx.failover().failover(ses, jobRes, new ArrayList<>(top), affKey, affCache);
 
-                        if (node == null) {
-                            String msg = "Failed to failover a job to another node (failover SPI returned null) [job=" +
-                                jobRes.getJob() + ", node=" + jobRes.getNode() + ']';
+            if (node == null) {
+                String msg = "Failed to failover a job to another node (failover SPI returned null) [job=" +
+                    jobRes.getJob() + ", node=" + jobRes.getNode() + ']';
 
-                            if (log.isDebugEnabled())
-                                log.debug(msg);
+                if (log.isDebugEnabled())
+                    log.debug(msg);
 
-                            Throwable e = new ClusterTopologyCheckedException(msg, jobRes.getException());
+                Throwable e = new ClusterTopologyCheckedException(msg, jobRes.getException());
 
-                            finishTask(null, e);
+                finishTask(null, e);
 
-                            return false;
-                        }
+                return false;
+            }
 
-                        if (log.isDebugEnabled())
-                            log.debug("Resolved job failover [newNode=" + node + ", oldNode=" + jobRes.getNode() +
-                                ", job=" + jobRes.getJob() + ", resMsg=" + res + ']');
+            if (log.isDebugEnabled())
+                log.debug("Resolved job failover [newNode=" + node + ", oldNode=" + jobRes.getNode() +
+                    ", job=" + jobRes.getJob() + ", resMsg=" + res + ']');
 
-                        jobRes.setNode(node);
-                        jobRes.resetResponse();
+            jobRes.setNode(node);
+            jobRes.resetResponse();
 
-                        if (!resCache) // Store result back in map before sending.
-                            GridTaskWorker.this.jobRes.put(res.getJobId(), jobRes);
+            if (!resCache) {
+                synchronized (mux) {
+                    // Store result back in map before sending.
+                    this.jobRes.put(res.getJobId(), jobRes);
+                }
+            }
 
-                        return true;
-                    }
-                    // Catch Throwable to protect against bad user code.
-                    catch (Throwable e) {
-                        String errMsg = "Failed to failover job due to undeclared user exception [job=" +
-                            jobRes.getJob() + ", err=" + e + ']';
+            return true;
+        }
+        // Catch Throwable to protect against bad user code.
+        catch (Throwable e) {
+            String errMsg = "Failed to failover job due to undeclared user exception [job=" +
+                jobRes.getJob() + ", err=" + e + ']';
 
-                        U.error(log, errMsg, e);
+            U.error(log, errMsg, e);
 
-                        finishTask(null, new ComputeUserUndeclaredException(errMsg, e));
+            finishTask(null, new ComputeUserUndeclaredException(errMsg, e));
 
-                        if (e instanceof Error)
-                            throw (Error)e;
+            if (e instanceof Error)
+                throw (Error)e;
 
-                        return false;
-                    }
-                }
-            }
-        });
+            return false;
+        }
     }
 
     /**


[36/50] ignite git commit: IGNITE-3153 TcpDiscoveryZookeeperIpFinder doesn't properly handle client reconnections

Posted by vo...@apache.org.
IGNITE-3153 TcpDiscoveryZookeeperIpFinder doesn't properly handle client reconnections


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

Branch: refs/heads/ignite-3341
Commit: 7da0a02a8aface7db7ac2485f22572f85ca67674
Parents: a4bbf0b
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Jun 16 10:49:48 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Jun 16 10:49:48 2016 +0300

----------------------------------------------------------------------
 .../zk/TcpDiscoveryZookeeperIpFinder.java       | 65 +++++++++++---------
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  | 20 ++++--
 2 files changed, 51 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7da0a02a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
index bee4dab..238987b 100644
--- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
@@ -39,6 +39,7 @@ import org.apache.curator.x.discovery.details.JsonInstanceSerializer;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinderAdapter;
@@ -68,11 +69,8 @@ import org.codehaus.jackson.map.annotate.JsonRootName;
  *
  * @see <a href="http://zookeeper.apache.org">Apache ZooKeeper</a>
  * @see <a href="http://curator.apache.org">Apache Curator</a>
- *
- * @author Raul Kripalani
  */
 public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
-
     /** System property name to provide the ZK Connection String. */
     public static final String PROP_ZK_CONNECTION_STRING = "IGNITE_ZK_CONNECTION_STRING";
 
@@ -89,6 +87,10 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
     @GridToStringExclude
     private final AtomicBoolean initGuard = new AtomicBoolean();
 
+    /** Init guard. */
+    @GridToStringExclude
+    private final AtomicBoolean closeGuard = new AtomicBoolean();
+
     /** Logger. */
     @LoggerResource
     private IgniteLogger log;
@@ -140,9 +142,11 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
             curator = CuratorFrameworkFactory.newClient(zkConnectionString, retryPolicy);
         }
 
-        if (curator.getState() != CuratorFrameworkState.STARTED)
+        if (curator.getState() == CuratorFrameworkState.LATENT)
             curator.start();
 
+        A.ensure(curator.getState() == CuratorFrameworkState.STARTED, "CuratorFramework can't be started.");
+
         discovery = ServiceDiscoveryBuilder.builder(IgniteInstanceDetails.class)
             .client(curator)
             .basePath(basePath)
@@ -152,8 +156,11 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
 
     /** {@inheritDoc} */
     @Override public void onSpiContextDestroyed() {
-        if (!initGuard.compareAndSet(true, false))
+        if (!closeGuard.compareAndSet(false, true)) {
+            U.warn(log, "ZooKeeper IP Finder can't be closed more than once.");
+
             return;
+        }
 
         log.info("Destroying ZooKeeper IP Finder.");
 
@@ -175,7 +182,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
 
         try {
             serviceInstances = discovery.queryForInstances(serviceName);
-        } catch (Exception e) {
+        }
+        catch (Exception e) {
             log.warning("Error while getting registered addresses from ZooKeeper IP Finder.", e);
             return Collections.emptyList();
         }
@@ -214,17 +222,18 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
 
             try {
                 ServiceInstance<IgniteInstanceDetails> si = ServiceInstance.<IgniteInstanceDetails>builder()
-                        .name(serviceName)
-                        .uriSpec(URI_SPEC)
-                        .address(addr.getAddress().getHostAddress())
-                        .port(addr.getPort())
-                        .build();
+                    .name(serviceName)
+                    .uriSpec(URI_SPEC)
+                    .address(addr.getAddress().getHostAddress())
+                    .port(addr.getPort())
+                    .build();
 
                 ourInstances.put(addr, si);
 
                 discovery.registerService(si);
 
-            } catch (Exception e) {
+            }
+            catch (Exception e) {
                 log.warning(String.format("Error while registering an address from ZooKeeper IP Finder " +
                     "[message=%s,addresses=%s]", e.getMessage(), addr), e);
             }
@@ -245,13 +254,14 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
             ServiceInstance<IgniteInstanceDetails> si = ourInstances.get(addr);
             if (si == null) {
                 log.warning("Asked to unregister address from ZooKeeper IP Finder, but no match was found in local " +
-                        "instance map for: " + addrs);
+                    "instance map for: " + addrs);
                 continue;
             }
 
             try {
                 discovery.unregisterService(si);
-            } catch (Exception e) {
+            }
+            catch (Exception e) {
                 log.warning("Error while unregistering an address from ZooKeeper IP Finder: " + addr, e);
             }
         }
@@ -272,7 +282,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
     }
 
     /**
-     * @param zkConnectionString ZooKeeper connection string in case a {@link CuratorFramework} is not being set explicitly.
+     * @param zkConnectionString ZooKeeper connection string in case a {@link CuratorFramework} is not being set
+     * explicitly.
      */
     public void setZkConnectionString(String zkConnectionString) {
         this.zkConnectionString = zkConnectionString;
@@ -286,8 +297,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
     }
 
     /**
-     * @param retryPolicy {@link RetryPolicy} to use in case a ZK Connection String is being injected, or if
-     *                    using a system property.
+     * @param retryPolicy {@link RetryPolicy} to use in case a ZK Connection String is being injected, or if using a
+     * system property.
      */
     public void setRetryPolicy(RetryPolicy retryPolicy) {
         this.retryPolicy = retryPolicy;
@@ -315,9 +326,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
     }
 
     /**
-     * @param serviceName Service name to use, as defined by Curator's {#link ServiceDiscovery} recipe. In physical
-     *                    ZK terms, it represents the node under {@link #basePath}, under which services will be
-     *                    registered.
+     * @param serviceName Service name to use, as defined by Curator's {#link ServiceDiscovery} recipe. In physical ZK
+     * terms, it represents the node under {@link #basePath}, under which services will be registered.
      */
     public void setServiceName(String serviceName) {
         this.serviceName = serviceName;
@@ -331,20 +341,19 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
     }
 
     /**
-     * @param allowDuplicateRegistrations Whether to register each node only once, or if duplicate registrations
-     *                                    are allowed. Nodes will attempt to register themselves, plus those they
-     *                                    know about. By default, duplicate registrations are not allowed, but you
-     *                                    might want to set this property to <tt>true</tt> if you have multiple
-     *                                    network interfaces or if you are facing troubles.
+     * @param allowDuplicateRegistrations Whether to register each node only once, or if duplicate registrations are
+     * allowed. Nodes will attempt to register themselves, plus those they know about. By default, duplicate
+     * registrations are not allowed, but you might want to set this property to <tt>true</tt> if you have multiple
+     * network interfaces or if you are facing troubles.
      */
     public void setAllowDuplicateRegistrations(boolean allowDuplicateRegistrations) {
         this.allowDuplicateRegistrations = allowDuplicateRegistrations;
     }
 
     /**
-     * Empty DTO for storing service instances details. Currently acting as a placeholder because Curator requires
-     * a payload type when registering and discovering nodes. May be enhanced in the future with further information
-     * to assist discovery.
+     * Empty DTO for storing service instances details. Currently acting as a placeholder because Curator requires a
+     * payload type when registering and discovering nodes. May be enhanced in the future with further information to
+     * assist discovery.
      *
      * @author Raul Kripalani
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7da0a02a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
index ec868ea..42f7a1d 100644
--- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
@@ -32,8 +32,10 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -42,7 +44,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  * @author Raul Kripalani
  */
 public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
-
     /** ZK Cluster size. */
     private static final int ZK_CLUSTER_SIZE = 3;
 
@@ -121,9 +122,9 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
         // first node => configure with zkUrl; second node => configure with CuratorFramework; third and subsequent
         // shall be configured through system property
-        if (gridName.equals(getTestGridName(0))) {
+        if (gridName.equals(getTestGridName(0)))
             zkIpFinder.setZkConnectionString(zkCluster.getConnectString());
-        }
+
         else if (gridName.equals(getTestGridName(1))) {
             zkIpFinder.setCurator(CuratorFrameworkFactory.newClient(zkCluster.getConnectString(),
                 new ExponentialBackoffRetry(100, 5)));
@@ -361,10 +362,17 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
         // stop all grids
         stopAllGrids();
-        Thread.sleep(2000);
 
-        // check that all nodes are gone in ZK
-        assertEquals(0, zkCurator.getChildren().forPath(SERVICES_IGNITE_ZK_PATH).size());
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return 0 == zkCurator.getChildren().forPath(SERVICES_IGNITE_ZK_PATH).size();
+                }
+                catch (Exception e) {
+                    return false;
+                }
+            }
+        }, 20000));
     }
 
     /**


[20/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
new file mode 100644
index 0000000..5da6ba2
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
@@ -0,0 +1,369 @@
+/*
+ * 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.tests;
+
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.pojos.Person;
+import org.apache.ignite.tests.pojos.PersonId;
+import org.apache.ignite.tests.utils.CacheStoreHelper;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.springframework.core.io.ClassPathResource;
+
+/**
+ * Unit tests for Ignite caches which utilizing {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}
+ * to store cache data into Cassandra tables
+ */
+public class IgnitePersistentStoreTest {
+    /** */
+    private static final Logger LOGGER = Logger.getLogger(IgnitePersistentStoreTest.class.getName());
+
+    /** */
+    @BeforeClass
+    public static void setUpClass() {
+        if (CassandraHelper.useEmbeddedCassandra()) {
+            try {
+                CassandraHelper.startEmbeddedCassandra(LOGGER);
+            }
+            catch (Throwable e) {
+                throw new RuntimeException("Failed to start embedded Cassandra instance", e);
+            }
+        }
+
+        LOGGER.info("Testing admin connection to Cassandra");
+        CassandraHelper.testAdminConnection();
+
+        LOGGER.info("Testing regular connection to Cassandra");
+        CassandraHelper.testRegularConnection();
+
+        LOGGER.info("Dropping all artifacts from previous tests execution session");
+        CassandraHelper.dropTestKeyspaces();
+
+        LOGGER.info("Start tests execution");
+    }
+
+    /** */
+    @AfterClass
+    public static void tearDownClass() {
+        try {
+            CassandraHelper.dropTestKeyspaces();
+        }
+        finally {
+            CassandraHelper.releaseCassandraResources();
+
+            if (CassandraHelper.useEmbeddedCassandra()) {
+                try {
+                    CassandraHelper.stopEmbeddedCassandra();
+                }
+                catch (Throwable e) {
+                    LOGGER.error("Failed to stop embedded Cassandra instance", e);
+                }
+            }
+        }
+    }
+
+    /** */
+    @Test
+    public void primitiveStrategyTest() {
+        Ignition.stopAll(true);
+
+        Map<Long, Long> longMap = TestsHelper.generateLongsMap();
+        Map<String, String> strMap = TestsHelper.generateStringsMap();
+
+        LOGGER.info("Running PRIMITIVE strategy write tests");
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/primitive/ignite-config.xml")) {
+            IgniteCache<Long, Long> longCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Long>("cache1"));
+            IgniteCache<String, String> strCache = ignite.getOrCreateCache(new CacheConfiguration<String, String>("cache2"));
+
+            LOGGER.info("Running single operation write tests");
+            longCache.put(1L, 1L);
+            strCache.put("1", "1");
+            LOGGER.info("Single operation write tests passed");
+
+            LOGGER.info("Running bulk operation write tests");
+            longCache.putAll(longMap);
+            strCache.putAll(strMap);
+            LOGGER.info("Bulk operation write tests passed");
+        }
+
+        LOGGER.info("PRIMITIVE strategy write tests passed");
+
+        Ignition.stopAll(true);
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/primitive/ignite-config.xml")) {
+            LOGGER.info("Running PRIMITIVE strategy read tests");
+
+            IgniteCache<Long, Long> longCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Long>("cache1"));
+            IgniteCache<String, String> strCache = ignite.getOrCreateCache(new CacheConfiguration<String, String>("cache2"));
+
+            LOGGER.info("Running single operation read tests");
+
+            Long longVal = longCache.get(1L);
+            if (!longVal.equals(longMap.get(1L)))
+                throw new RuntimeException("Long value was incorrectly deserialized from Cassandra");
+
+            String strVal = strCache.get("1");
+            if (!strVal.equals(strMap.get("1")))
+                throw new RuntimeException("String value was incorrectly deserialized from Cassandra");
+
+            LOGGER.info("Single operation read tests passed");
+
+            LOGGER.info("Running bulk operation read tests");
+
+            Map<Long, Long> longMap1 = longCache.getAll(longMap.keySet());
+            if (!TestsHelper.checkMapsEqual(longMap, longMap1))
+                throw new RuntimeException("Long values batch was incorrectly deserialized from Cassandra");
+
+            Map<String, String> strMap1 = strCache.getAll(strMap.keySet());
+            if (!TestsHelper.checkMapsEqual(strMap, strMap1))
+                throw new RuntimeException("String values batch was incorrectly deserialized from Cassandra");
+
+            LOGGER.info("Bulk operation read tests passed");
+
+            LOGGER.info("PRIMITIVE strategy read tests passed");
+
+            LOGGER.info("Running PRIMITIVE strategy delete tests");
+
+            longCache.remove(1L);
+            longCache.removeAll(longMap.keySet());
+
+            strCache.remove("1");
+            strCache.removeAll(strMap.keySet());
+
+            LOGGER.info("PRIMITIVE strategy delete tests passed");
+        }
+    }
+
+    /** */
+    @Test
+    public void blobStrategyTest() {
+        Ignition.stopAll(true);
+
+        Map<Long, Long> longMap = TestsHelper.generateLongsMap();
+        Map<Long, Person> personMap = TestsHelper.generateLongsPersonsMap();
+
+        LOGGER.info("Running BLOB strategy write tests");
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/blob/ignite-config.xml")) {
+            IgniteCache<Long, Long> longCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Long>("cache1"));
+            IgniteCache<Long, Person> personCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Person>("cache2"));
+
+            LOGGER.info("Running single operation write tests");
+            longCache.put(1L, 1L);
+            personCache.put(1L, TestsHelper.generateRandomPerson());
+            LOGGER.info("Single operation write tests passed");
+
+            LOGGER.info("Running bulk operation write tests");
+            longCache.putAll(longMap);
+            personCache.putAll(personMap);
+            LOGGER.info("Bulk operation write tests passed");
+        }
+
+        LOGGER.info("BLOB strategy write tests passed");
+
+        Ignition.stopAll(true);
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/blob/ignite-config.xml")) {
+            LOGGER.info("Running BLOB strategy read tests");
+
+            IgniteCache<Long, Long> longCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Long>("cache1"));
+            IgniteCache<Long, Person> personCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Person>("cache2"));
+
+            LOGGER.info("Running single operation read tests");
+
+            Long longVal = longCache.get(1L);
+            if (!longVal.equals(longMap.get(1L)))
+                throw new RuntimeException("Long value was incorrectly deserialized from Cassandra");
+
+            Person person = personCache.get(1L);
+            if (!person.equals(personMap.get(1L)))
+                throw new RuntimeException("Person value was incorrectly deserialized from Cassandra");
+
+            LOGGER.info("Single operation read tests passed");
+
+            LOGGER.info("Running bulk operation read tests");
+
+            Map<Long, Long> longMap1 = longCache.getAll(longMap.keySet());
+            if (!TestsHelper.checkMapsEqual(longMap, longMap1))
+                throw new RuntimeException("Long values batch was incorrectly deserialized from Cassandra");
+
+            Map<Long, Person> personMap1 = personCache.getAll(personMap.keySet());
+            if (!TestsHelper.checkPersonMapsEqual(personMap, personMap1, false))
+                throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra");
+
+            LOGGER.info("Bulk operation read tests passed");
+
+            LOGGER.info("BLOB strategy read tests passed");
+
+            LOGGER.info("Running BLOB strategy delete tests");
+
+            longCache.remove(1L);
+            longCache.removeAll(longMap.keySet());
+
+            personCache.remove(1L);
+            personCache.removeAll(personMap.keySet());
+
+            LOGGER.info("BLOB strategy delete tests passed");
+        }
+    }
+
+    /** */
+    @Test
+    public void pojoStrategyTest() {
+        Ignition.stopAll(true);
+
+        LOGGER.info("Running POJO strategy write tests");
+
+        Map<Long, Person> personMap1 = TestsHelper.generateLongsPersonsMap();
+        Map<PersonId, Person> personMap2 = TestsHelper.generatePersonIdsPersonsMap();
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) {
+            IgniteCache<Long, Person> personCache1 = ignite.getOrCreateCache(new CacheConfiguration<Long, Person>("cache1"));
+            IgniteCache<PersonId, Person> personCache2 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache2"));
+            IgniteCache<PersonId, Person> personCache3 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache3"));
+
+            LOGGER.info("Running single operation write tests");
+            personCache1.put(1L, TestsHelper.generateRandomPerson());
+            personCache2.put(TestsHelper.generateRandomPersonId(), TestsHelper.generateRandomPerson());
+            personCache3.put(TestsHelper.generateRandomPersonId(), TestsHelper.generateRandomPerson());
+            LOGGER.info("Single operation write tests passed");
+
+            LOGGER.info("Running bulk operation write tests");
+            personCache1.putAll(personMap1);
+            personCache2.putAll(personMap2);
+            personCache3.putAll(personMap2);
+            LOGGER.info("Bulk operation write tests passed");
+        }
+
+        LOGGER.info("POJO strategy write tests passed");
+
+        Ignition.stopAll(true);
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) {
+            LOGGER.info("Running POJO strategy read tests");
+
+            IgniteCache<Long, Person> personCache1 = ignite.getOrCreateCache(new CacheConfiguration<Long, Person>("cache1"));
+            IgniteCache<PersonId, Person> personCache2 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache2"));
+            IgniteCache<PersonId, Person> personCache3 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache3"));
+
+            LOGGER.info("Running single operation read tests");
+            Person person = personCache1.get(1L);
+            if (!person.equalsPrimitiveFields(personMap1.get(1L)))
+                throw new RuntimeException("Person value was incorrectly deserialized from Cassandra");
+
+            PersonId id = personMap2.keySet().iterator().next();
+
+            person = personCache2.get(id);
+            if (!person.equalsPrimitiveFields(personMap2.get(id)))
+                throw new RuntimeException("Person value was incorrectly deserialized from Cassandra");
+
+            person = personCache3.get(id);
+            if (!person.equals(personMap2.get(id)))
+                throw new RuntimeException("Person value was incorrectly deserialized from Cassandra");
+
+            LOGGER.info("Single operation read tests passed");
+
+            LOGGER.info("Running bulk operation read tests");
+
+            Map<Long, Person> persons1 = personCache1.getAll(personMap1.keySet());
+            if (!TestsHelper.checkPersonMapsEqual(persons1, personMap1, true))
+                throw new RuntimeException("Persons values batch was incorrectly deserialized from Cassandra");
+
+            Map<PersonId, Person> persons2 = personCache2.getAll(personMap2.keySet());
+            if (!TestsHelper.checkPersonMapsEqual(persons2, personMap2, true))
+                throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra");
+
+            Map<PersonId, Person> persons3 = personCache3.getAll(personMap2.keySet());
+            if (!TestsHelper.checkPersonMapsEqual(persons3, personMap2, false))
+                throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra");
+
+            LOGGER.info("Bulk operation read tests passed");
+
+            LOGGER.info("POJO strategy read tests passed");
+
+            LOGGER.info("Running POJO strategy delete tests");
+
+            personCache1.remove(1L);
+            personCache1.removeAll(personMap1.keySet());
+
+            personCache2.remove(id);
+            personCache2.removeAll(personMap2.keySet());
+
+            personCache3.remove(id);
+            personCache3.removeAll(personMap2.keySet());
+
+            LOGGER.info("POJO strategy delete tests passed");
+        }
+    }
+
+    /** */
+    @Test
+    public void loadCacheTest() {
+        Ignition.stopAll(true);
+
+        LOGGER.info("Running loadCache test");
+
+        LOGGER.info("Filling Cassandra table with test data");
+
+        CacheStore store = CacheStoreHelper.createCacheStore("personTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        Collection<CacheEntryImpl<PersonId, Person>> entries = TestsHelper.generatePersonIdsPersonsEntries();
+
+        store.writeAll(entries);
+
+        LOGGER.info("Cassandra table filled with test data");
+
+        LOGGER.info("Running loadCache test");
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) {
+            IgniteCache<PersonId, Person> personCache3 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache3"));
+            int size = personCache3.size(CachePeekMode.ALL);
+
+            LOGGER.info("Initial cache size " + size);
+
+            LOGGER.info("Loading cache data from Cassandra table");
+
+            personCache3.loadCache(null, new String[] {"select * from test1.pojo_test3 limit 3"});
+
+            size = personCache3.size(CachePeekMode.ALL);
+            if (size != 3) {
+                throw new RuntimeException("Cache data was incorrectly loaded from Cassandra. " +
+                    "Expected number of records is 3, but loaded number of records is " + size);
+            }
+
+            LOGGER.info("Cache data loaded from Cassandra table");
+        }
+
+        LOGGER.info("loadCache test passed");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java
new file mode 100644
index 0000000..0c18bc0
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java
@@ -0,0 +1,27 @@
+/*
+ * 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.tests.load;
+
+/**
+ * Generator abstraction which could be used by tests to generate next key/value pair for Ignite cache
+ * from provided int number (which sequentially incremented in load test driver loop).
+ */
+public interface Generator {
+    /** */
+    public Object generate(long i);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java
new file mode 100644
index 0000000..a31abee
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.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.tests.load;
+
+/**
+ * Implementation of {@link org.apache.ignite.tests.load.Generator} generating {@link Integer} instance.
+ */
+public class IntGenerator implements Generator {
+    /** {@inheritDoc} */
+    @Override public Object generate(long i) {
+        long val = i / 10000;
+
+        while (val > Integer.MAX_VALUE)
+            val = val / 2;
+
+        return (int)val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java
new file mode 100644
index 0000000..296839d
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java
@@ -0,0 +1,238 @@
+/*
+ * 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.tests.load;
+
+import java.lang.reflect.Constructor;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.cassandra.common.SystemHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.log4j.Logger;
+
+/**
+ * Basic load test driver to be inherited by specific implementation for particular use-case.
+ */
+public abstract class LoadTestDriver {
+    /** Number of attempts to setup load test */
+    private static final int NUMBER_OF_SETUP_ATTEMPTS = 10;
+
+    /** Timeout between load test setup attempts */
+    private static final int SETUP_ATTEMPT_TIMEOUT = 1000;
+
+    /** */
+    public void runTest(String testName, Class<? extends Worker> clazz, String logName) {
+        logger().info("Running " + testName + " test");
+
+        Object cfg = null;
+
+        int attempt;
+
+        logger().info("Setting up load tests driver");
+
+        for (attempt = 0; attempt < NUMBER_OF_SETUP_ATTEMPTS; attempt++) {
+            try {
+                cfg = setup(logName);
+                break;
+            }
+            catch (Throwable e) {
+                logger().error((attempt + 1) + " attempt to setup load test '" + testName + "' failed", e);
+            }
+
+            if (attempt + 1 != NUMBER_OF_SETUP_ATTEMPTS) {
+                logger().info("Sleeping for " + SETUP_ATTEMPT_TIMEOUT + " seconds before trying next attempt " +
+                        "to setup '" + testName + "' load test");
+
+                try {
+                    Thread.sleep(SETUP_ATTEMPT_TIMEOUT);
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+            }
+        }
+
+        if (cfg == null && attempt == NUMBER_OF_SETUP_ATTEMPTS) {
+            throw new RuntimeException("All " + NUMBER_OF_SETUP_ATTEMPTS + " attempts to setup load test '" +
+                    testName+ "' have failed");
+        }
+
+        // calculates host unique prefix based on its subnet IP address
+        long hostUniqePrefix = getHostUniquePrefix();
+
+        logger().info("Load tests driver setup successfully completed");
+
+        try {
+
+            List<Worker> workers = new LinkedList<>();
+            long startPosition = 0;
+
+            logger().info("Starting workers");
+
+            for (int i = 0; i < TestsHelper.getLoadTestsThreadsCount(); i++) {
+                Worker worker = createWorker(clazz, cfg,
+                    hostUniqePrefix + startPosition,
+                    hostUniqePrefix + startPosition + 100000000);
+                workers.add(worker);
+                worker.setName(testName + "-worker-" + i);
+                worker.start();
+                startPosition += 100000001;
+            }
+
+            logger().info("Workers started");
+            logger().info("Waiting for workers to complete");
+
+            List<String> failedWorkers = new LinkedList<>();
+
+            for (Worker worker : workers) {
+                boolean failed = false;
+
+                try {
+                    worker.join();
+                }
+                catch (Throwable e) {
+                    logger().error("Worker " + worker.getName() + " waiting interrupted", e);
+                    failed = true;
+                }
+
+                if (failed || worker.isFailed()) {
+                    failedWorkers.add(worker.getName());
+                    logger().info("Worker " + worker.getName() + " execution failed");
+                }
+                else
+                    logger().info("Worker " + worker.getName() + " successfully completed");
+            }
+
+            printTestResultsHeader(testName, failedWorkers);
+            printTestResultsStatistics(testName, workers);
+        }
+        finally {
+            tearDown(cfg);
+        }
+    }
+
+    /** */
+    protected abstract Logger logger();
+
+    /** */
+    protected abstract Object setup(String logName);
+
+    /** */
+    protected void tearDown(Object obj) {
+    }
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private Worker createWorker(Class clazz, Object cfg, long startPosition, long endPosition) {
+        try {
+            Class cfgCls = cfg instanceof Ignite ? Ignite.class : CacheStore.class;
+
+            Constructor ctor = clazz.getConstructor(cfgCls, long.class, long.class);
+
+            return (Worker)ctor.newInstance(cfg, startPosition, endPosition);
+        }
+        catch (Throwable e) {
+            logger().error("Failed to instantiate worker of class '" + clazz.getName() + "'", e);
+            throw new RuntimeException("Failed to instantiate worker of class '" + clazz.getName() + "'", e);
+        }
+    }
+
+    /** */
+    private void printTestResultsHeader(String testName, List<String> failedWorkers) {
+        if (failedWorkers.isEmpty()) {
+            logger().info(testName + " test execution successfully completed.");
+            return;
+        }
+
+        if (failedWorkers.size() == TestsHelper.getLoadTestsThreadsCount()) {
+            logger().error(testName + " test execution totally failed.");
+            return;
+        }
+
+        String strFailedWorkers = "";
+
+        for (String workerName : failedWorkers) {
+            if (!strFailedWorkers.isEmpty())
+                strFailedWorkers += ", ";
+
+            strFailedWorkers += workerName;
+        }
+
+        logger().warn(testName + " test execution completed, but " + failedWorkers.size() + " of " +
+            TestsHelper.getLoadTestsThreadsCount() + " workers failed. Failed workers: " + strFailedWorkers);
+    }
+
+    /** */
+    @SuppressWarnings("StringBufferReplaceableByString")
+    private void printTestResultsStatistics(String testName, List<Worker> workers) {
+        long cnt = 0;
+        long errCnt = 0;
+        long speed = 0;
+
+        for (Worker worker : workers) {
+            cnt += worker.getMsgProcessed();
+            errCnt += worker.getErrorsCount();
+            speed += worker.getSpeed();
+        }
+
+        float errPercent = errCnt == 0 ?
+            0 :
+            cnt + errCnt ==  0 ? 0 : (float)(errCnt * 100 ) / (float)(cnt + errCnt);
+
+        StringBuilder builder = new StringBuilder();
+        builder.append(SystemHelper.LINE_SEPARATOR);
+        builder.append("-------------------------------------------------");
+        builder.append(SystemHelper.LINE_SEPARATOR);
+        builder.append(testName).append(" test statistics").append(SystemHelper.LINE_SEPARATOR);
+        builder.append(testName).append(" messages: ").append(cnt).append(SystemHelper.LINE_SEPARATOR);
+        builder.append(testName).append(" errors: ").append(errCnt).append(", ").
+                append(String.format("%.2f", errPercent).replace(",", ".")).
+                append("%").append(SystemHelper.LINE_SEPARATOR);
+        builder.append(testName).append(" speed: ").append(speed).append(" msg/sec").append(SystemHelper.LINE_SEPARATOR);
+        builder.append("-------------------------------------------------");
+
+        logger().info(builder.toString());
+    }
+
+    /** */
+    private long getHostUniquePrefix() {
+        String[] parts = SystemHelper.HOST_IP.split("\\.");
+
+        if (parts[2].equals("0"))
+            parts[2] = "777";
+
+        if (parts[3].equals("0"))
+            parts[3] = "777";
+
+        long part3 = Long.parseLong(parts[2]);
+        long part4 = Long.parseLong(parts[3]);
+
+        if (part3 < 10)
+            part3 = part3 * 100;
+        else if (part4 < 100)
+            part3 = part3 * 10;
+
+        if (part4 < 10)
+            part4 = part4 * 100;
+        else if (part4 < 100)
+            part4 = part4 * 10;
+
+        return (part4 * 100000000000000L) + (part3 * 100000000000L) + Thread.currentThread().getId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LongGenerator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LongGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LongGenerator.java
new file mode 100644
index 0000000..0398f98
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LongGenerator.java
@@ -0,0 +1,28 @@
+/*
+ * 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.tests.load;
+
+/**
+ * Implementation of {@link org.apache.ignite.tests.load.Generator} generating {@link Long} instance.
+ */
+public class LongGenerator implements Generator {
+    /** {@inheritDoc} */
+    @Override public Object generate(long i) {
+        return i;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java
new file mode 100644
index 0000000..0317320
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java
@@ -0,0 +1,43 @@
+/*
+ * 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.tests.load;
+
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.tests.pojos.Person;
+
+/**
+ * Implementation of {@link Generator} generating {@link Person} instance.
+ */
+public class PersonGenerator implements Generator {
+    /** */
+    private static final Date DATE = new Date();
+
+    /** */
+    private static final List<String> PHONES = new LinkedList<String>(){{
+        add("1234567");
+        add("7654321");
+        add("1289054");
+    }};
+
+    /** {@inheritDoc} */
+    @Override public Object generate(long i) {
+        return new Person(Long.toString(i), Long.toString(i), (int)(i % 100), i % 2 == 0, i, i, DATE, PHONES);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java
new file mode 100644
index 0000000..a11e0d8
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java
@@ -0,0 +1,31 @@
+/*
+ * 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.tests.load;
+
+import org.apache.ignite.tests.pojos.PersonId;
+
+/**
+ * Implementation of {@link org.apache.ignite.tests.load.Generator} generating
+ * {@link org.apache.ignite.tests.pojos.PersonId} instance.
+ */
+public class PersonIdGenerator implements Generator {
+    /** {@inheritDoc} */
+    @Override public Object generate(long i) {
+        return new PersonId(Long.toString(i), Long.toString(i), i);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java
new file mode 100644
index 0000000..cfaf34a
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java
@@ -0,0 +1,28 @@
+/*
+ * 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.tests.load;
+
+/**
+ * Implementation of {@link org.apache.ignite.tests.load.Generator} generating {@link String} instance.
+ */
+public class StringGenerator implements Generator {
+    /** {@inheritDoc} */
+    @Override public Object generate(long i) {
+        return Long.toString(i);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java
new file mode 100644
index 0000000..f4bffc7
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java
@@ -0,0 +1,429 @@
+/*
+ * 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.tests.load;
+
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.cassandra.common.SystemHelper;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.log4j.Logger;
+
+/**
+ * Worker thread abstraction to be inherited by specific load test implementation
+ */
+public abstract class Worker extends Thread {
+    /** */
+    private static final SimpleDateFormat TIME_FORMATTER = new SimpleDateFormat("hh:mm:ss");
+
+    /** */
+    private long testStartTime;
+
+    /** */
+    boolean warmup = TestsHelper.getLoadTestsWarmupPeriod() != 0;
+
+    /** */
+    private volatile long warmupStartTime = 0;
+
+    /** */
+    private volatile long warmupFinishTime = 0;
+
+    /** */
+    private volatile long startTime = 0;
+
+    /** */
+    private volatile long finishTime = 0;
+
+    /** */
+    private volatile long warmupMsgProcessed = 0;
+
+    /** */
+    private volatile long warmupSleepCnt = 0;
+
+    /** */
+    private volatile long msgProcessed = 0;
+
+    /** */
+    private volatile long msgFailed = 0;
+
+    /** */
+    private volatile long sleepCnt = 0;
+
+    /** */
+    private Throwable executionError;
+
+    /** */
+    private long statReportedTime;
+
+    /** */
+    private CacheStore cacheStore;
+
+    /** */
+    private Ignite ignite;
+
+    /** */
+    private IgniteCache igniteCache;
+
+    /** */
+    private Logger log;
+
+    /** */
+    private long startPosition;
+
+    /** */
+    private long endPosition;
+
+    /** */
+    public Worker(CacheStore cacheStore, long startPosition, long endPosition) {
+        this.cacheStore = cacheStore;
+        this.log = Logger.getLogger(loggerName());
+        this.startPosition = startPosition;
+        this.endPosition = endPosition;
+    }
+
+    /** */
+    public Worker(Ignite ignite, long startPosition, long endPosition) {
+        this.ignite = ignite;
+        this.log = Logger.getLogger(loggerName());
+        this.startPosition = startPosition;
+        this.endPosition = endPosition;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void run() {
+        try {
+            if (ignite != null)
+                igniteCache = ignite.getOrCreateCache(new CacheConfiguration(TestsHelper.getLoadTestsCacheName()));
+
+            execute();
+        }
+        catch (Throwable e) {
+            executionError = e;
+            throw new RuntimeException("Test execution abnormally terminated", e);
+        }
+        finally {
+            reportTestCompletion();
+        }
+    }
+
+    /** */
+    public boolean isFailed() {
+        return executionError != null;
+    }
+
+    /** */
+    public long getSpeed() {
+        if (msgProcessed == 0)
+            return 0;
+
+        long finish = finishTime != 0 ? finishTime : System.currentTimeMillis();
+        long duration = (finish - startTime - sleepCnt * TestsHelper.getLoadTestsRequestsLatency()) / 1000;
+
+        return duration == 0 ? msgProcessed : msgProcessed / duration;
+    }
+
+    /** */
+    public long getErrorsCount() {
+        return msgFailed;
+    }
+
+    /** */
+    public float getErrorsPercent() {
+        if (msgFailed == 0)
+            return 0;
+
+        return msgProcessed + msgFailed == 0 ? 0 : (float)(msgFailed * 100 ) / (float)(msgProcessed + msgFailed);
+    }
+
+    /** */
+    public long getMsgCountTotal() {
+        return warmupMsgProcessed + msgProcessed;
+    }
+
+    /** */
+    public long getWarmupMsgProcessed() {
+        return warmupMsgProcessed;
+    }
+
+    /** */
+    public long getMsgProcessed() {
+        return msgProcessed;
+    }
+
+    /** */
+    protected abstract String loggerName();
+
+    /** */
+    protected abstract boolean batchMode();
+
+    /** */
+    protected void process(CacheStore cacheStore, CacheEntryImpl entry) {
+        throw new UnsupportedOperationException("Single message processing is not supported");
+    }
+
+    /** */
+    protected void process(IgniteCache cache, Object key, Object val) {
+        throw new UnsupportedOperationException("Single message processing is not supported");
+    }
+
+    /** */
+    protected void process(CacheStore cacheStore, Collection<CacheEntryImpl> entries) {
+        throw new UnsupportedOperationException("Batch processing is not supported");
+    }
+
+    /** */
+    protected void process(IgniteCache cache, Map map) {
+        throw new UnsupportedOperationException("Batch processing is not supported");
+    }
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private void execute() throws InterruptedException {
+        testStartTime = System.currentTimeMillis();
+
+        log.info("Test execution started");
+
+        if (warmup)
+            log.info("Warm up period started");
+
+        warmupStartTime = warmup ? testStartTime : 0;
+        startTime = !warmup ? testStartTime : 0;
+
+        statReportedTime = testStartTime;
+
+        long cntr = startPosition;
+        Object key = TestsHelper.generateLoadTestsKey(cntr);
+        Object val = TestsHelper.generateLoadTestsValue(cntr);
+        List<CacheEntryImpl> batchList = new ArrayList<>(TestsHelper.getBulkOperationSize());
+        Map batchMap = new HashMap(TestsHelper.getBulkOperationSize());
+
+        int execTime = TestsHelper.getLoadTestsWarmupPeriod() + TestsHelper.getLoadTestsExecutionTime();
+
+        try {
+            while (true) {
+                if (System.currentTimeMillis() - testStartTime > execTime)
+                    break;
+
+                if (warmup && System.currentTimeMillis() - testStartTime > TestsHelper.getLoadTestsWarmupPeriod()) {
+                    warmupFinishTime = System.currentTimeMillis();
+                    startTime = warmupFinishTime;
+                    statReportedTime = warmupFinishTime;
+                    warmup = false;
+                    log.info("Warm up period completed");
+                }
+
+                if (!batchMode()) {
+                    if (cacheStore != null)
+                        doWork(new CacheEntryImpl(key, val));
+                    else
+                        doWork(key, val);
+                }
+                else if (batchList.size() == TestsHelper.getBulkOperationSize() ||
+                    batchMap.size() == TestsHelper.getBulkOperationSize()) {
+                    if (cacheStore != null)
+                        doWork(batchList);
+                    else
+                        doWork(batchMap);
+
+                    batchMap.clear();
+                    batchList.clear();
+                }
+
+                if (cntr == endPosition)
+                    cntr = startPosition;
+                else
+                    cntr++;
+
+                key = TestsHelper.generateLoadTestsKey(cntr);
+                val = TestsHelper.generateLoadTestsValue(cntr);
+
+                if (batchMode()) {
+                    if (cacheStore != null)
+                        batchList.add(new CacheEntryImpl(key, val));
+                    else
+                        batchMap.put(key, val);
+                }
+
+                reportStatistics();
+            }
+        }
+        finally {
+            warmupFinishTime = warmupFinishTime != 0 ? warmupFinishTime : System.currentTimeMillis();
+            finishTime = System.currentTimeMillis();
+        }
+    }
+
+    /** */
+    private void doWork(CacheEntryImpl entry) {
+        try {
+            process(cacheStore, entry);
+            updateMetrics(1);
+        }
+        catch (Throwable e) {
+            log.error("Failed to perform single operation", e);
+            updateErrorMetrics(1);
+        }
+    }
+
+    /** */
+    private void doWork(Object key, Object val) {
+        try {
+            process(igniteCache, key, val);
+            updateMetrics(1);
+        }
+        catch (Throwable e) {
+            log.error("Failed to perform single operation", e);
+            updateErrorMetrics(1);
+        }
+    }
+
+    /** */
+    private void doWork(Collection<CacheEntryImpl> entries) {
+        try {
+            process(cacheStore, entries);
+            updateMetrics(entries.size());
+        }
+        catch (Throwable e) {
+            log.error("Failed to perform batch operation", e);
+            updateErrorMetrics(entries.size());
+        }
+    }
+
+    /** */
+    private void doWork(Map entries) {
+        try {
+            process(igniteCache, entries);
+            updateMetrics(entries.size());
+        }
+        catch (Throwable e) {
+            log.error("Failed to perform batch operation", e);
+            updateErrorMetrics(entries.size());
+        }
+    }
+
+    /** */
+    private long getWarmUpSpeed() {
+        if (warmupMsgProcessed == 0)
+            return 0;
+
+        long finish = warmupFinishTime != 0 ? warmupFinishTime : System.currentTimeMillis();
+        long duration = (finish - warmupStartTime - warmupSleepCnt * TestsHelper.getLoadTestsRequestsLatency()) / 1000;
+
+        return duration == 0 ? warmupMsgProcessed : warmupMsgProcessed / duration;
+    }
+
+    /** */
+    private void updateMetrics(int itemsProcessed) {
+        if (warmup)
+            warmupMsgProcessed += itemsProcessed;
+        else
+            msgProcessed += itemsProcessed;
+
+        if (TestsHelper.getLoadTestsRequestsLatency() > 0) {
+            try {
+                Thread.sleep(TestsHelper.getLoadTestsRequestsLatency());
+
+                if (warmup)
+                    warmupSleepCnt++;
+                else
+                    sleepCnt++;
+            }
+            catch (Throwable ignored) {
+            }
+        }
+    }
+
+    /**
+     * TODO IGNITE-1371 Comment absent.
+     *
+     * @param itemsFailed Failed item.
+     */
+    private void updateErrorMetrics(int itemsFailed) {
+        if (!warmup)
+            msgFailed += itemsFailed;
+    }
+
+    /** */
+    private void reportStatistics() {
+        // statistics should be reported only every 30 seconds
+        if (System.currentTimeMillis() - statReportedTime < 30000)
+            return;
+
+        statReportedTime = System.currentTimeMillis();
+
+        int completed = warmup ?
+                (int)(statReportedTime - warmupStartTime) * 100 / TestsHelper.getLoadTestsWarmupPeriod() :
+                (int)(statReportedTime - startTime) * 100 / TestsHelper.getLoadTestsExecutionTime();
+
+        if (completed > 100)
+            completed = 100;
+
+        if (warmup) {
+            log.info("Warm up messages processed " + warmupMsgProcessed + ", " +
+                "speed " + getWarmUpSpeed() + " msg/sec, " + completed + "% completed");
+        }
+        else {
+            log.info("Messages processed " + msgProcessed + ", " +
+                "speed " + getSpeed() + " msg/sec, " + completed + "% completed, " +
+                "errors " + msgFailed + " / " + String.format("%.2f", getErrorsPercent()).replace(",", ".") + "%");
+        }
+    }
+
+    /** */
+    private void reportTestCompletion() {
+        StringBuilder builder = new StringBuilder();
+
+        if (executionError != null)
+            builder.append("Test execution abnormally terminated. ");
+        else
+            builder.append("Test execution successfully completed. ");
+
+        builder.append("Statistics: ").append(SystemHelper.LINE_SEPARATOR);
+        builder.append("Start time: ").append(TIME_FORMATTER.format(testStartTime)).append(SystemHelper.LINE_SEPARATOR);
+        builder.append("Finish time: ").append(TIME_FORMATTER.format(finishTime)).append(SystemHelper.LINE_SEPARATOR);
+        builder.append("Duration: ").append((finishTime - testStartTime) / 1000).append(" sec")
+            .append(SystemHelper.LINE_SEPARATOR);
+
+        if (TestsHelper.getLoadTestsWarmupPeriod() > 0) {
+            builder.append("Warm up period: ").append(TestsHelper.getLoadTestsWarmupPeriod() / 1000)
+                .append(" sec").append(SystemHelper.LINE_SEPARATOR);
+            builder.append("Warm up processed messages: ").append(warmupMsgProcessed).append(SystemHelper.LINE_SEPARATOR);
+            builder.append("Warm up processing speed: ").append(getWarmUpSpeed())
+                .append(" msg/sec").append(SystemHelper.LINE_SEPARATOR);
+        }
+
+        builder.append("Processed messages: ").append(msgProcessed).append(SystemHelper.LINE_SEPARATOR);
+        builder.append("Processing speed: ").append(getSpeed()).append(" msg/sec").append(SystemHelper.LINE_SEPARATOR);
+        builder.append("Errors: ").append(msgFailed).append(" / ").
+                append(String.format("%.2f", getErrorsPercent()).replace(",", ".")).append("%");
+
+        if (executionError != null)
+            log.error(builder.toString(), executionError);
+        else
+            log.info(builder.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java
new file mode 100644
index 0000000..38f0db8
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java
@@ -0,0 +1,63 @@
+/*
+ * 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.tests.load.cassandra;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.load.Worker;
+import org.apache.ignite.tests.utils.TestsHelper;
+
+/**
+ * Cassandra direct load tests worker for bulk read operation CacheStore.load
+ */
+public class BulkReadWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "CassandraBulkReadLoadTest";
+
+    /** */
+    private List<Object> keys = new ArrayList<>(TestsHelper.getBulkOperationSize());
+
+    /** */
+    public BulkReadWorker(CacheStore cacheStore, long startPosition, long endPosition) {
+        super(cacheStore, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(CacheStore cacheStore, Collection<CacheEntryImpl> entries) {
+        keys.clear();
+
+        for (CacheEntryImpl entry : entries)
+            keys.add(entry.getKey());
+
+        cacheStore.loadAll(keys);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java
new file mode 100644
index 0000000..c71728f
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tests.load.cassandra;
+
+import java.util.Collection;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Cassandra direct load tests worker for bulk write operation CacheStore.writeAll
+ */
+public class BulkWriteWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "CassandraBulkWriteLoadTest";
+
+    /** */
+    public BulkWriteWorker(CacheStore cacheStore, long startPosition, long endPosition) {
+        super(cacheStore, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(CacheStore cacheStore, Collection<CacheEntryImpl> entries) {
+        cacheStore.writeAll(entries);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java
new file mode 100644
index 0000000..051b55f
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tests.load.cassandra;
+
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Cassandra direct load tests worker for read operation CacheStore.load
+ */
+public class ReadWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "CassandraReadLoadTest";
+
+    /** */
+    public ReadWorker(CacheStore cacheStore, long startPosition, long endPosition) {
+        super(cacheStore, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(CacheStore cacheStore, CacheEntryImpl entry) {
+        cacheStore.load(entry.getKey());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java
new file mode 100644
index 0000000..2b10bcd
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tests.load.cassandra;
+
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Cassandra direct load tests worker for write operation CacheStore.write
+ */
+public class WriteWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "CassandraWriteLoadTest";
+
+    /** */
+    public WriteWorker(CacheStore cacheStore, long startPosition, long endPosition) {
+        super(cacheStore, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(CacheStore cacheStore, CacheEntryImpl entry) {
+        cacheStore.write(entry);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java
new file mode 100644
index 0000000..c20d0ce
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tests.load.ignite;
+
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Ignite load tests worker for bulk read operation CacheStore.loadAll
+ */
+public class BulkReadWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "IgniteBulkReadLoadTest";
+
+    /** */
+    public BulkReadWorker(Ignite ignite, long startPosition, long endPosition) {
+        super(ignite, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(IgniteCache cache, Map entries) {
+        cache.getAll(entries.keySet());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java
new file mode 100644
index 0000000..1ce7be3
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tests.load.ignite;
+
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Ignite load tests worker for bulk read operation CacheStore.writeAll
+ */
+public class BulkWriteWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "IgniteBulkWriteLoadTest";
+
+    /** */
+    public BulkWriteWorker(Ignite ignite, long startPosition, long endPosition) {
+        super(ignite, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(IgniteCache cache, Map entries) {
+        cache.putAll(entries);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java
new file mode 100644
index 0000000..35f7d39
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tests.load.ignite;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Ignite load tests worker for read operation CacheStore.load
+ */
+public class ReadWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "IgniteReadLoadTest";
+
+    /** */
+    public ReadWorker(Ignite ignite, long startPosition, long endPosition) {
+        super(ignite, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(IgniteCache cache, Object key, Object val) {
+        cache.get(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java
new file mode 100644
index 0000000..bed7099
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tests.load.ignite;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.tests.load.Worker;
+
+/**
+ * Ignite load tests worker for write operation CacheStore.write
+ */
+public class WriteWorker extends Worker {
+    /** */
+    public static final String LOGGER_NAME = "IgniteWriteLoadTest";
+
+    /** */
+    public WriteWorker(Ignite ignite, long startPosition, long endPosition) {
+        super(ignite, startPosition, endPosition);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String loggerName() {
+        return LOGGER_NAME;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean batchMode() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected void process(IgniteCache cache, Object key, Object val) {
+        cache.put(key, val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java
new file mode 100644
index 0000000..8a1e623
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java
@@ -0,0 +1,246 @@
+/*
+ * 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.tests.pojos;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Date;
+import java.util.List;
+
+/**
+ * Simple POJO which could be stored as a value in Ignite cache
+ */
+public class Person implements Externalizable {
+    /** */
+    private String firstName;
+
+    /** */
+    private String lastName;
+
+    /** */
+    private int age;
+
+    /** */
+    private boolean married;
+
+    /** */
+    private long height;
+
+    /** */
+    private float weight;
+
+    /** */
+    private Date birthDate;
+
+    /** */
+    private List<String> phones;
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public Person() {
+    }
+
+    /** */
+    public Person(String firstName, String lastName, int age, boolean married,
+        long height, float weight, Date birthDate, List<String> phones) {
+        this.firstName = firstName;
+        this.lastName = lastName;
+        this.age = age;
+        this.married = married;
+        this.height = height;
+        this.weight = weight;
+        this.birthDate = birthDate;
+        this.phones = phones;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(firstName);
+        out.writeObject(lastName);
+        out.writeInt(age);
+        out.writeBoolean(married);
+        out.writeLong(height);
+        out.writeFloat(weight);
+        out.writeObject(birthDate);
+        out.writeObject(phones);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        firstName = (String)in.readObject();
+        lastName = (String)in.readObject();
+        age = in.readInt();
+        married = in.readBoolean();
+        height = in.readLong();
+        weight = in.readFloat();
+        birthDate = (Date)in.readObject();
+        phones = (List<String>)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("SimplifiableIfStatement")
+    @Override public boolean equals(Object obj) {
+        if (obj == null || !(obj instanceof Person))
+            return false;
+
+        Person person = (Person)obj;
+
+        if ((firstName != null && !firstName.equals(person.firstName)) ||
+            (person.firstName != null && !person.firstName.equals(firstName)))
+            return false;
+
+        if ((lastName != null && !lastName.equals(person.lastName)) ||
+            (person.lastName != null && !person.lastName.equals(lastName)))
+            return false;
+
+        if ((birthDate != null && !birthDate.equals(person.birthDate)) ||
+            (person.birthDate != null && !person.birthDate.equals(birthDate)))
+            return false;
+
+        if ((phones != null && !phones.equals(person.phones)) ||
+            (person.phones != null && !person.phones.equals(phones)))
+            return false;
+
+        return age == person.age && married == person.married &&
+            height == person.height && weight == person.weight;
+    }
+
+    /** */
+    @SuppressWarnings("SimplifiableIfStatement")
+    public boolean equalsPrimitiveFields(Object obj) {
+        if (obj == null || !(obj instanceof Person))
+            return false;
+
+        Person person = (Person)obj;
+
+        if ((firstName != null && !firstName.equals(person.firstName)) ||
+            (person.firstName != null && !person.firstName.equals(firstName)))
+            return false;
+
+        if ((lastName != null && !lastName.equals(person.lastName)) ||
+            (person.lastName != null && !person.lastName.equals(lastName)))
+            return false;
+
+        if ((birthDate != null && !birthDate.equals(person.birthDate)) ||
+            (person.birthDate != null && !person.birthDate.equals(birthDate)))
+            return false;
+
+        return age == person.age && married == person.married &&
+            height == person.height && weight == person.weight;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setFirstName(String name) {
+        firstName = name;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public String getFirstName() {
+        return firstName;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setLastName(String name) {
+        lastName = name;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public String getLastName() {
+        return lastName;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setAge(int age) {
+        this.age = age;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public int getAge() {
+        return age;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setMarried(boolean married) {
+        this.married = married;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public boolean getMarried() {
+        return married;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setHeight(long height) {
+        this.height = height;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public long getHeight() {
+        return height;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setWeight(float weight) {
+        this.weight = weight;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public float getWeight() {
+        return weight;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setBirthDate(Date date) {
+        birthDate = date;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public Date getBirthDate() {
+        return birthDate;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setPhones(List<String> phones) {
+        this.phones = phones;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public List<String> getPhones() {
+        return phones;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.java
new file mode 100644
index 0000000..0dd5ab8
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.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.tests.pojos;
+
+import java.io.Serializable;
+
+/**
+ * Simple POJO which could be stored as a key in Ignite cache
+ */
+public class PersonId implements Serializable {
+    /** */
+    private String companyCode;
+
+    /** */
+    private String departmentCode;
+
+    /** */
+    private long personNum;
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public PersonId() {
+    }
+
+    /** */
+    public PersonId(String companyCode, String departmentCode, long personNum) {
+        this.companyCode = companyCode;
+        this.departmentCode = departmentCode;
+        this.personNum = personNum;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("SimplifiableIfStatement")
+    @Override public boolean equals(Object obj) {
+        if (obj == null || !(obj instanceof PersonId))
+            return false;
+
+        PersonId id = (PersonId)obj;
+
+        if ((companyCode != null && !companyCode.equals(id.companyCode)) ||
+            (id.companyCode != null && !id.companyCode.equals(companyCode)))
+            return false;
+
+        if ((companyCode != null && !companyCode.equals(id.companyCode)) ||
+            (id.companyCode != null && !id.companyCode.equals(companyCode)))
+            return false;
+
+        return personNum == id.personNum;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        String code = (companyCode == null ? "" : companyCode) +
+            (departmentCode == null ? "" : departmentCode) +
+                personNum;
+
+        return code.hashCode();
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setCompanyCode(String code) {
+        companyCode = code;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public String getCompanyCode() {
+        return companyCode;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setDepartmentCode(String code) {
+        departmentCode = code;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public String getDepartmentCode() {
+        return departmentCode;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setPersonNumber(long personNum) {
+        this.personNum = personNum;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public long getPersonNumber() {
+        return personNum;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
new file mode 100644
index 0000000..b5ff5ad
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
@@ -0,0 +1,64 @@
+/*
+ * 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.tests.utils;
+
+import java.lang.reflect.Field;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.cassandra.CassandraCacheStore;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.logger.log4j.Log4JLogger;
+import org.apache.log4j.Logger;
+import org.springframework.core.io.Resource;
+
+/**
+ * Helper class utilized by unit tests to get appropriate instance of {@link CacheStore}
+ */
+public class CacheStoreHelper {
+    /** */
+    private static final Logger LOGGER = Logger.getLogger(CacheStoreHelper.class.getName());
+
+    /** */
+    public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource conn) {
+        return createCacheStore(cacheName, persistenceSettings, conn, LOGGER);
+    }
+
+    /** */
+    public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource conn,
+        Logger log) {
+        CassandraCacheStore<Integer, Integer> cacheStore =
+            new CassandraCacheStore<>(conn, new KeyValuePersistenceSettings(persistenceSettings),
+                Runtime.getRuntime().availableProcessors());
+
+        try {
+            Field sesField = CassandraCacheStore.class.getDeclaredField("storeSes");
+            Field logField = CassandraCacheStore.class.getDeclaredField("log");
+
+            sesField.setAccessible(true);
+            logField.setAccessible(true);
+
+            sesField.set(cacheStore, new TestCacheSession(cacheName));
+            logField.set(cacheStore, new Log4JLogger(log));
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to initialize test Ignite cache store", e);
+        }
+
+        return cacheStore;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java
new file mode 100644
index 0000000..66df6e7
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tests.utils;
+
+import org.apache.ignite.cache.store.cassandra.datasource.Credentials;
+
+/**
+ * Implementation of {@link org.apache.ignite.cache.store.cassandra.datasource.Credentials}
+ * providing admin user/password to establish Cassandra session.
+ */
+public class CassandraAdminCredentials implements Credentials {
+    /** {@inheritDoc} */
+    @Override public String getUser() {
+        return CassandraHelper.getAdminUser();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getPassword() {
+        return CassandraHelper.getAdminPassword();
+    }
+}


[13/50] ignite git commit: IGNITE-3273 - fixed

Posted by vo...@apache.org.
IGNITE-3273 - fixed


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

Branch: refs/heads/ignite-3341
Commit: e27ba14d51b9ba43a1f5a467d09b2e5e5072cd9c
Parents: 5f44672
Author: Sergi Vladykin <se...@gmail.com>
Authored: Tue Jun 14 21:47:11 2016 +0300
Committer: Sergi Vladykin <se...@gmail.com>
Committed: Wed Jun 15 10:28:49 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/sql/GridSqlConst.java   |  5 ++
 .../processors/query/h2/sql/GridSqlJoin.java    | 17 +++--
 .../processors/query/h2/sql/GridSqlType.java    |  5 ++
 .../query/IgniteSqlSplitterSelfTest.java        | 75 ++++++++++++++++++++
 4 files changed, 93 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e27ba14d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
index 36c95ce..976eb2c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlConst.java
@@ -19,12 +19,17 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.Collections;
 import org.h2.value.Value;
+import org.h2.value.ValueBoolean;
 
 /**
  * Constant value.
  */
 public class GridSqlConst extends GridSqlElement implements GridSqlValue {
     /** */
+    public static final GridSqlElement TRUE = new GridSqlConst(ValueBoolean.get(true))
+        .resultType(GridSqlType.BOOLEAN);
+
+    /** */
     private final Value val;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e27ba14d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
index 0148404..f1ad2e5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlJoin.java
@@ -36,13 +36,15 @@ public class GridSqlJoin extends GridSqlElement {
      * @param on Join condition.
      */
     public GridSqlJoin(GridSqlElement leftTbl, GridSqlElement rightTbl, boolean leftOuter, @Nullable GridSqlElement on) {
-        super(new ArrayList<GridSqlElement>(on == null ? 2 : 3));
+        super(new ArrayList<GridSqlElement>(3));
 
         addChild(leftTbl);
         addChild(rightTbl);
 
-        if (on != null)
-            addChild(on);
+        if (on == null) // To avoid query nesting issues in FROM clause we need to always generate ON condition.
+            on = GridSqlConst.TRUE;
+
+        addChild(on);
 
         this.leftOuter = leftOuter;
     }
@@ -64,8 +66,8 @@ public class GridSqlJoin extends GridSqlElement {
     /**
      * @return {@code JOIN ON} condition.
      */
-    @Nullable public GridSqlElement on() {
-        return size() < 3 ? null : child(2);
+    public GridSqlElement on() {
+        return child(2);
     }
 
     /** {@inheritDoc} */
@@ -78,10 +80,7 @@ public class GridSqlJoin extends GridSqlElement {
 
         buff.append(rightTable().getSQL());
 
-        GridSqlElement on = on();
-
-        if (on != null)
-            buff.append(" \n ON ").append(StringUtils.unEnclose(on.getSQL()));
+        buff.append(" \n ON ").append(StringUtils.unEnclose(on().getSQL()));
 
         return buff.toString();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e27ba14d/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
index febf174..efe9138 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlType.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.h2.expression.Expression;
 import org.h2.table.Column;
 import org.h2.value.Value;
+import org.h2.value.ValueBoolean;
 import org.h2.value.ValueDouble;
 import org.h2.value.ValueLong;
 
@@ -43,6 +44,10 @@ public final class GridSqlType {
     /** */
     public static final GridSqlType UUID = new GridSqlType(Value.UUID, 0, Integer.MAX_VALUE, 36, "UUID");
 
+    /** */
+    public static final GridSqlType BOOLEAN = new GridSqlType(Value.BOOLEAN, 0, ValueBoolean.PRECISION,
+        ValueBoolean.DISPLAY_SIZE, "BOOLEAN");
+
     /** H2 type. */
     private final int type;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e27ba14d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index d0e2780..fd52469 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -313,6 +313,81 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testImplicitJoinConditionGeneration() {
+        IgniteCache<Integer, Person> p = ignite(0).createCache(cacheConfig("P", true, Integer.class, Person.class));
+        IgniteCache<Integer, Department> d = ignite(0).createCache(cacheConfig("D", true, Integer.class, Department.class));
+        IgniteCache<Integer, Org> o = ignite(0).createCache(cacheConfig("O", true, Integer.class, Org.class));
+
+        try {
+            info("Plan: " + p.query(new SqlFieldsQuery(
+                "explain select P.Person.*,dep.*,org.* " +
+                    "from P.Person inner join D.Department dep ON dep.id=P.Person.depId " +
+                    "left join O.Org org ON org.id=dep.orgId"
+            )).getAll());
+
+            assertEquals(0, p.query(new SqlFieldsQuery(
+                "select P.Person.*,dep.*,org.* " +
+                    "from P.Person inner join D.Department dep ON dep.id=P.Person.depId " +
+                    "left join O.Org org ON org.id=dep.orgId"
+            )).getAll().size());
+        }
+        finally {
+            p.destroy();
+            d.destroy();
+            o.destroy();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class Person {
+        /** */
+        @QuerySqlField
+        private int id;
+
+        /** */
+        @QuerySqlField
+        private String name;
+
+        /** */
+        @QuerySqlField
+        private int depId;
+    }
+
+    /**
+     *
+     */
+    public static class Org {
+        /** */
+        @QuerySqlField(index = true)
+        private int id;
+
+        /** */
+        @QuerySqlField
+        private String name;
+    }
+
+    /**
+     *
+     */
+    public static class Department {
+        /** */
+        @QuerySqlField(index = true)
+        private int id;
+
+        /** */
+        @QuerySqlField(index = true)
+        private int orgId;
+
+        /** */
+        @QuerySqlField
+        private String name;
+    }
+
+    /**
      * Test value.
      */
     private static class GroupIndexTestValue implements Serializable {


[39/50] ignite git commit: Fixed JavaDocs

Posted by vo...@apache.org.
Fixed JavaDocs


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

Branch: refs/heads/ignite-3341
Commit: ee4165626478c6df7a5db79bbe37b1893ab0ce6f
Parents: 706a935
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Jun 16 16:37:44 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Jun 16 16:37:44 2016 +0300

----------------------------------------------------------------------
 .../cassandra/datasource/package-info.java      | 21 ++++++++++++++++++++
 .../cache/store/cassandra/package-info.java     | 21 ++++++++++++++++++++
 .../cassandra/persistence/package-info.java     | 21 ++++++++++++++++++++
 .../cassandra/serializer/package-info.java      | 21 ++++++++++++++++++++
 .../store/cassandra/utils/package-info.java     | 21 ++++++++++++++++++++
 pom.xml                                         |  2 +-
 6 files changed, 106 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee416562/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java
new file mode 100644
index 0000000..9ec05b2
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains data source abstraction to specify configuration of the Cassandra session.
+ */
+package org.apache.ignite.cache.store.cassandra.datasource;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee416562/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java
new file mode 100644
index 0000000..a859f60
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains Cassandra-based implementation of persistent store.
+ */
+package org.apache.ignite.cache.store.cassandra;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee416562/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java
new file mode 100644
index 0000000..891469d
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains persistence setting classes.
+ */
+package org.apache.ignite.cache.store.cassandra.persistence;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee416562/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java
new file mode 100644
index 0000000..31a5282
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains serializers for Cassandra persistent store.
+ */
+package org.apache.ignite.cache.store.cassandra.serializer;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee416562/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java
new file mode 100644
index 0000000..11fb039
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Contains DDL generator.
+ */
+package org.apache.ignite.cache.store.cassandra.utils;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee416562/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0761aec..89ceb62 100644
--- a/pom.xml
+++ b/pom.xml
@@ -184,7 +184,7 @@
                                     <reportOutputDirectory>${basedir}/target/javadoc</reportOutputDirectory>
                                     <destDir>core</destDir>
                                     <stylesheetfile>${basedir}/assembly/docfiles/javadoc.css</stylesheetfile>
-                                    <subpackages>org.apache.ignite -exclude org.apache.ignite.client:org.apache.ignite.codegen:org.apache.ignite.examples:org.apache.ignite.internal:org.apache.ignite.schema:org.apache.ignite.tests:org.apache.ignite.tools:org.apache.ignite.util:org.apache.ignite.spi.discovery.tcp.messages:org.apache.ignite.spi.discovery.tcp.internal:org.apache.ignite.spi.deployment.uri.scanners:org.apache.ignite.spi.deployment.uri.tasks:org.apache.ignite.yardstick</subpackages>
+                                    <subpackages>org.apache.ignite -exclude org.apache.ignite.client:org.apache.ignite.codegen:org.apache.ignite.examples:org.apache.ignite.internal:org.apache.ignite.schema:org.apache.ignite.tests:org.apache.ignite.tools:org.apache.ignite.util:org.apache.ignite.spi.discovery.tcp.messages:org.apache.ignite.spi.discovery.tcp.internal:org.apache.ignite.spi.deployment.uri.scanners:org.apache.ignite.spi.deployment.uri.tasks:org.apache.ignite.yardstick:org.apache.ignite.cache.store.cassandra.common:org.apache.ignite.cache.store.cassandra.session:org.apache.ignite.cache.store.cassandra.session.pool</subpackages>
                                 </configuration>
                             </execution>
                         </executions>


[16/50] ignite git commit: IGNITE-3215 - Added IgniteRDD.withKeepBinary method (ScalaDoc)

Posted by vo...@apache.org.
IGNITE-3215 - Added IgniteRDD.withKeepBinary method (ScalaDoc)


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

Branch: refs/heads/ignite-3341
Commit: e235298f9bc2c895e17170b57dd8ad5f90cc25e8
Parents: 0e18941
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Jun 15 12:19:44 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Jun 15 12:19:44 2016 +0300

----------------------------------------------------------------------
 .../src/main/scala/org/apache/ignite/spark/IgniteRDD.scala    | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e235298f/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index cad96b9..fb15ff1 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -32,6 +32,7 @@ import org.apache.spark.sql._
 import org.apache.spark._
 
 import scala.collection.JavaConversions._
+import scala.reflect.macros.whitebox
 
 /**
  * Ignite RDD. Represents Ignite cache as Spark RDD abstraction.
@@ -293,6 +294,12 @@ class IgniteRDD[K, V] (
         ensureCache().removeAll()
     }
 
+    /**
+     * Returns `IgniteRDD` that will operate with binary objects. This method
+     * behaves similar to [[org.apache.ignite.IgniteCache#withKeepBinary]].
+     *
+     * @return New `IgniteRDD` instance for binary objects.
+     */
     def withKeepBinary[K1, V1](): IgniteRDD[K1, V1] = {
         new IgniteRDD[K1, V1](
             ic.asInstanceOf[IgniteContext[K1, V1]],


[41/50] ignite git commit: IGNITE-3331: IGFS: Better affinity management for client tasks.

Posted by vo...@apache.org.
IGNITE-3331: IGFS: Better affinity management for client tasks.


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

Branch: refs/heads/ignite-3341
Commit: 06831b13f4a9693e767f2e299abcc57605c87da5
Parents: f7da0dd
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 13:27:18 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 13:27:18 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 45 ++++++++++++++++----
 .../igfs/client/IgfsClientAbstractCallable.java |  5 +++
 .../igfs/client/IgfsClientAffinityCallable.java |  5 +++
 .../igfs/client/IgfsClientDeleteCallable.java   |  5 +++
 .../igfs/client/IgfsClientExistsCallable.java   |  5 +++
 .../igfs/client/IgfsClientInfoCallable.java     |  5 +++
 .../client/IgfsClientListFilesCallable.java     |  5 +++
 .../client/IgfsClientListPathsCallable.java     |  5 +++
 .../igfs/client/IgfsClientMkdirsCallable.java   |  5 +++
 .../igfs/client/IgfsClientRenameCallable.java   |  5 +++
 .../igfs/client/IgfsClientSetTimesCallable.java |  5 +++
 .../igfs/client/IgfsClientSizeCallable.java     |  5 +++
 .../igfs/client/IgfsClientSummaryCallable.java  |  5 +++
 .../igfs/client/IgfsClientUpdateCallable.java   |  5 +++
 .../meta/IgfsClientMetaIdsForPathCallable.java  |  5 +++
 .../meta/IgfsClientMetaInfoForPathCallable.java |  5 +++
 16 files changed, 111 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 943c4c8..30c7d10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -150,8 +150,11 @@ public class IgfsMetaManager extends IgfsManager {
     /** Client flag. */
     private final boolean client;
 
+    /** Compute facade. */
+    private IgniteCompute compute;
+
     /** Compute facade for client tasks. */
-    private IgniteCompute cliCompute;
+    private IgniteCompute metaCompute;
 
     /**
      * Constructor.
@@ -248,7 +251,12 @@ public class IgfsMetaManager extends IgfsManager {
      */
     <T> T runClientTask(IgfsClientAbstractCallable<T> task) {
         try {
-            return clientCompute().call(task);
+            if (!task.isReadOnly() && cfg.isColocateMetadata())
+                // If task mutates state and co-location is enabled, we route request to primary node.
+                return compute().affinityCall(cfg.getMetaCacheName(), IgfsUtils.ROOT_ID, task);
+            else
+                // Otherwise we route to any available data node.
+                return metaCompute().call(task);
         }
         catch (ClusterTopologyException e) {
             throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e);
@@ -260,24 +268,43 @@ public class IgfsMetaManager extends IgfsManager {
      *
      * @return Compute facade.
      */
-    private IgniteCompute clientCompute() {
+    private IgniteCompute compute() {
+        IgniteCompute compute0 = compute;
+
+        if (compute0 == null) {
+            compute0 = igfsCtx.kernalContext().grid().compute();
+
+            compute = compute0;
+        }
+
+        assert compute0 != null;
+
+        return compute0;
+    }
+
+    /**
+     * Get metadata compute facade for client tasks.
+     *
+     * @return Metadata compute facade.
+     */
+    private IgniteCompute metaCompute() {
         assert client;
 
-        IgniteCompute cliCompute0 = cliCompute;
+        IgniteCompute metaCompute0 = metaCompute;
 
-        if (cliCompute0 == null) {
+        if (metaCompute0 == null) {
             IgniteEx ignite = igfsCtx.kernalContext().grid();
 
             ClusterGroup cluster = ignite.cluster().forIgfsMetadataDataNodes(cfg.getName(), cfg.getMetaCacheName());
 
-            cliCompute0 = ignite.compute(cluster);
+            metaCompute0 = ignite.compute(cluster);
 
-            cliCompute = cliCompute0;
+            metaCompute = metaCompute0;
         }
 
-        assert cliCompute0 != null;
+        assert metaCompute0 != null;
 
-        return cliCompute0;
+        return metaCompute0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
index d9c3456..c06619f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -85,6 +85,11 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
      */
     protected abstract T call0(IgfsContext ctx) throws Exception;
 
+    /**
+     * @return {@code True} if task is read-only and does not change metadata.
+     */
+    public abstract boolean isReadOnly();
+
     /** {@inheritDoc} */
     @Override public final void writeBinary(BinaryWriter writer) throws BinaryObjectException {
         BinaryRawWriter rawWriter = writer.rawWriter();

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
index 1668f36..6e2748d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
@@ -74,6 +74,11 @@ public class IgfsClientAffinityCallable extends IgfsClientAbstractCallable<Colle
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeLong(start);
         writer.writeLong(len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
index c1b8be8..21f8c46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
@@ -61,6 +61,11 @@ public class IgfsClientDeleteCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeBoolean(recursive);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
index 04b63d8..536cdc8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
@@ -52,6 +52,11 @@ public class IgfsClientExistsCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientExistsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
index f97c3c4..6bab8b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
@@ -53,6 +53,11 @@ public class IgfsClientInfoCallable extends IgfsClientAbstractCallable<IgfsFile>
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientInfoCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
index 325e714..deb8836 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
@@ -55,6 +55,11 @@ public class IgfsClientListFilesCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListFilesCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
index 78b4c84..ad6c858 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
@@ -54,6 +54,11 @@ public class IgfsClientListPathsCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListPathsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
index 944da6f..6ecb6ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -66,6 +66,11 @@ public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         IgfsUtils.writeProperties(writer, props);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
index 55afb83..142af47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
@@ -64,6 +64,11 @@ public class IgfsClientRenameCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         IgfsUtils.writePath(writer, destPath);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
index 277effc..8c302bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
@@ -69,6 +69,11 @@ public class IgfsClientSetTimesCallable extends IgfsClientAbstractCallable<Void>
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeLong(accessTime);
         writer.writeLong(modificationTime);

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
index 474a940..29a7b7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
@@ -53,6 +53,11 @@ public class IgfsClientSizeCallable extends IgfsClientAbstractCallable<Long> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSizeCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
index 7e29029..1e2795c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
@@ -53,6 +53,11 @@ public class IgfsClientSummaryCallable extends IgfsClientAbstractCallable<IgfsPa
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSummaryCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
index 4acf4eb..82d6e8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -65,6 +65,11 @@ public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFil
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         IgfsUtils.writeProperties(writer, props);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
index 7b3d142..b07a0cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
@@ -59,6 +59,11 @@ public class IgfsClientMetaIdsForPathCallable extends IgfsClientAbstractCallable
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaIdsForPathCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/06831b13/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
index cb31663..e7e0164 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
@@ -57,6 +57,11 @@ public class IgfsClientMetaInfoForPathCallable extends IgfsClientAbstractCallabl
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaInfoForPathCallable.class, this);
     }


[03/50] ignite git commit: ignite-114 Load value from store for cache 'invoke'

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughStoreCallTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughStoreCallTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughStoreCallTest.java
new file mode 100644
index 0000000..bb092d4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheReadThroughStoreCallTest.java
@@ -0,0 +1,288 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteRunnable;
+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.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionIsolation.values;
+
+/**
+ *
+ */
+public class IgniteCacheReadThroughStoreCallTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final Map<Object, Object> storeMap = new ConcurrentHashMap8<>();
+
+    /** */
+    protected boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        storeMap.clear();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultiNode() throws Exception {
+        startGridsMultiThreaded(4);
+
+        client = true;
+
+        startGrid(4);
+
+        checkLoadCount(cacheConfiguration(PARTITIONED, ATOMIC, 0));
+
+        checkLoadCount(cacheConfiguration(PARTITIONED, ATOMIC, 1));
+
+        checkLoadCount(cacheConfiguration(PARTITIONED, ATOMIC, 2));
+
+        checkLoadCount(cacheConfiguration(PARTITIONED, TRANSACTIONAL, 0));
+
+        checkLoadCount(cacheConfiguration(PARTITIONED, TRANSACTIONAL, 1));
+
+        checkLoadCount(cacheConfiguration(PARTITIONED, TRANSACTIONAL, 2));
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void checkLoadCount(CacheConfiguration<Object, Object> ccfg) throws Exception {
+        storeMap.clear();
+
+        Ignite ignite0 = ignite(0);
+
+        ignite0.createCache(ccfg);
+
+        try {
+            int key = 0;
+
+            for (Ignite node : G.allGrids()) {
+                log.info("Test for node: " + node.name());
+
+                final IgniteCache<Object, Object> cache = node.cache(ccfg.getName());
+
+                for (int i = 0; i < 50; i++) {
+                    final int k = key++;
+
+                    checkReadThrough(cache, new IgniteRunnable() {
+                        @Override public void run() {
+                            cache.invoke(k, new TestEntryProcessor());
+                        }
+                    }, null, null, 1);
+                }
+
+                for (int i = 0; i < 50; i++) {
+                    final int k = key++;
+
+                    checkReadThrough(cache, new IgniteRunnable() {
+                        @Override public void run() {
+                            cache.put(k, k);
+                        }
+                    }, null, null, 0);
+                }
+
+                if (ccfg.getAtomicityMode() == TRANSACTIONAL) {
+                    for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+                        for (TransactionIsolation isolation : values()) {
+                            log.info("Test tx [concurrency=" + concurrency + ", isolation=" + isolation + ']');
+
+                            for (int i = 0; i < 50; i++) {
+                                final int k = key++;
+
+                                checkReadThrough(cache, new IgniteRunnable() {
+                                    @Override public void run() {
+                                        cache.invoke(k, new TestEntryProcessor());
+                                    }
+                                }, concurrency, isolation, 2);
+                            }
+                        }
+                    }
+                }
+            }
+
+            ignite0.cache(ccfg.getName()).removeAll();
+        }
+        finally {
+            ignite0.destroyCache(ccfg.getName());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param c Cache operation Closure.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @param expLoadCnt Expected number of store 'load' calls.
+     * @throws Exception If failed.
+     */
+    private void checkReadThrough(IgniteCache<Object, Object> cache,
+        IgniteRunnable c,
+        @Nullable TransactionConcurrency concurrency,
+        @Nullable TransactionIsolation isolation,
+        int expLoadCnt) throws Exception {
+        TestStore.loadCnt.set(0);
+
+        Transaction tx = isolation != null ? cache.unwrap(Ignite.class).transactions().txStart(concurrency, isolation)
+            : null;
+
+        try {
+            c.run();
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assertEquals(expLoadCnt, TestStore.loadCnt.get());
+    }
+
+    /**
+     * @param cacheMode Cache mode.
+     * @param atomicityMode Atomicity mode.
+     * @param backups Number of backups.
+     * @return Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration<Object, Object> cacheConfiguration(CacheMode cacheMode,
+        CacheAtomicityMode atomicityMode,
+        int backups) {
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setReadThrough(true);
+        ccfg.setWriteThrough(true);
+        ccfg.setCacheStoreFactory(new TestStoreFactory());
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(backups);
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    public static class TestStoreFactory implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return new TestStore();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class TestStore extends CacheStoreAdapter<Object, Object> {
+        /** */
+        static AtomicInteger loadCnt = new AtomicInteger();
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Object, Object> clo, Object... args) {
+            fail();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object load(Object key) {
+            loadCnt.incrementAndGet();
+
+            return storeMap.get(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> entry) {
+            storeMap.put(entry.getKey(), entry.getValue());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            storeMap.remove(key);
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestEntryProcessor implements EntryProcessor<Object, Object, Object> {
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... args) {
+            Object val = entry.getValue();
+
+            entry.setValue(entry.getKey());
+
+            return val;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
index 28a954b..dd6b268 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
@@ -147,6 +147,16 @@ public abstract class IgniteCacheLoaderWriterAbstractTest extends IgniteCacheAbs
         assertFalse(storeMap.containsKey(key));
 
         assertNull(cache.get(key));
+
+        ldrCallCnt.set(0);
+
+        cache.invoke(key, new EntryProcessor<Object, Object, Object>() {
+            @Override public Object process(MutableEntry<Object, Object> e, Object... args) {
+                return null;
+            }
+        });
+
+        checkCalls(1, 0);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 5a017e6..883f426 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -65,8 +65,10 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationDefau
 import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationTemplateTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDynamicStopSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughSingleNodeTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheInvokeReadThroughTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLoadRebalanceEvictionSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughStoreCallTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxCopyOnReadDisabledTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxLocalPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxLocalStoreValueTest;
@@ -203,7 +205,9 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteCacheTxLocalPeekModesTest.class);
         suite.addTestSuite(IgniteCacheTxReplicatedPeekModesTest.class);
 
+        suite.addTestSuite(IgniteCacheInvokeReadThroughSingleNodeTest.class);
         suite.addTestSuite(IgniteCacheInvokeReadThroughTest.class);
+        suite.addTestSuite(IgniteCacheReadThroughStoreCallTest.class);
         suite.addTestSuite(GridCacheVersionMultinodeTest.class);
 
         suite.addTestSuite(IgniteCacheNearReadCommittedTest.class);


[02/50] ignite git commit: ignite-3261 Store information about AffinityKey class in metadata cache.

Posted by vo...@apache.org.
ignite-3261 Store information about AffinityKey class in metadata cache.


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

Branch: refs/heads/ignite-3341
Commit: 2b64e7c8b059f31e5ccd6a475b1ba0ad198d86fd
Parents: 1cb291c
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 14 09:45:10 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 14 09:45:10 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryContext.java   | 43 ++++++++++--
 .../ignite/internal/binary/AffinityKey.java     | 69 ++++++++++++++++++++
 .../binary/GridBinaryAffinityKeySelfTest.java   | 15 +++++
 ...aultBinaryMappersBinaryMetaDataSelfTest.java | 17 +++++
 4 files changed, 137 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2b64e7c8/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index daf34ad..d39f9c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -259,7 +259,7 @@ public class BinaryContext {
         registerPredefinedType(LinkedHashMap.class, 0);
 
         // Classes with overriden default serialization flag.
-        registerPredefinedType(AffinityKey.class, 0, affinityFieldName(AffinityKey.class));
+        registerPredefinedType(AffinityKey.class, 0, affinityFieldName(AffinityKey.class), false);
 
         registerPredefinedType(GridMapEntry.class, 60);
         registerPredefinedType(IgniteBiTuple.class, 61);
@@ -562,9 +562,37 @@ public class BinaryContext {
         if (desc == null)
             desc = registerClassDescriptor(cls, deserialize);
         else if (!desc.registered()) {
-            assert desc.userType();
-
-            desc = registerUserClassDescriptor(desc);
+            if (!desc.userType()) {
+                BinaryClassDescriptor desc0 = new BinaryClassDescriptor(
+                    this,
+                    desc.describedClass(),
+                    false,
+                    desc.typeId(),
+                    desc.typeName(),
+                    desc.affFieldKeyName(),
+                    desc.mapper(),
+                    desc.initialSerializer(),
+                    false,
+                    true
+                );
+
+                if (descByCls.replace(cls, desc, desc0)) {
+                    Collection<BinarySchema> schemas =
+                        desc0.schema() != null ? Collections.singleton(desc.schema()) : null;
+
+                    BinaryMetadata meta = new BinaryMetadata(desc0.typeId(),
+                        desc0.typeName(),
+                        desc0.fieldsMeta(),
+                        desc0.affFieldKeyName(),
+                        schemas, desc0.isEnum());
+
+                    metaHnd.addMeta(desc0.typeId(), meta.wrap(this));
+
+                    return desc0;
+                }
+            }
+            else
+                desc = registerUserClassDescriptor(desc);
         }
 
         return desc;
@@ -959,15 +987,16 @@ public class BinaryContext {
      * @return GridBinaryClassDescriptor.
      */
     public BinaryClassDescriptor registerPredefinedType(Class<?> cls, int id) {
-        return registerPredefinedType(cls, id, null);
+        return registerPredefinedType(cls, id, null, true);
     }
 
     /**
      * @param cls Class.
      * @param id Type ID.
+     * @param affFieldName Affinity field name.
      * @return GridBinaryClassDescriptor.
      */
-    public BinaryClassDescriptor registerPredefinedType(Class<?> cls, int id, String affFieldName) {
+    public BinaryClassDescriptor registerPredefinedType(Class<?> cls, int id, String affFieldName, boolean registered) {
         String simpleClsName = SIMPLE_NAME_LOWER_CASE_MAPPER.typeName(cls.getName());
 
         if (id == 0)
@@ -983,7 +1012,7 @@ public class BinaryContext {
             SIMPLE_NAME_LOWER_CASE_MAPPER,
             new BinaryReflectiveSerializer(),
             false,
-            true /* registered */
+            registered /* registered */
         );
 
         predefinedTypeNames.put(simpleClsName, id);

http://git-wip-us.apache.org/repos/asf/ignite/blob/2b64e7c8/modules/core/src/test/java/org/apache/ignite/internal/binary/AffinityKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/AffinityKey.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/AffinityKey.java
new file mode 100644
index 0000000..1b4daee
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/AffinityKey.java
@@ -0,0 +1,69 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class AffinityKey {
+    /** Key. */
+    private int key;
+
+    /** Affinity key. */
+    @AffinityKeyMapped
+    private int aff;
+
+    /**
+     * @param key Key.
+     * @param aff Affinity key.
+     */
+    public AffinityKey(int key, int aff) {
+        this.key = key;
+        this.aff = aff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        AffinityKey that = (AffinityKey) o;
+
+        return key == that.key && aff == that.aff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = key;
+
+        res = 31 * res + aff;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(AffinityKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2b64e7c8/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
index 06406e0..2b54f6b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryAffinityKeySelfTest.java
@@ -139,6 +139,8 @@ public class GridBinaryAffinityKeySelfTest extends GridCommonAbstractTest {
 
             assertEquals(i, aff.affinityKey(ignite.binary().toBinary(new TestObject(i))));
 
+            assertEquals(i, aff.affinityKey(new AffinityKey(0, i)));
+
             BinaryObjectBuilder bldr = ignite.binary().builder("TestObject2");
 
             bldr.setField("affKey", i);
@@ -162,6 +164,8 @@ public class GridBinaryAffinityKeySelfTest extends GridCommonAbstractTest {
             assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, new TestObject(i)));
 
             assertEquals(affProc.mapKeyToNode(null, i), affProc.mapKeyToNode(null, cacheObj));
+
+            assertEquals(affProc.mapKeyToNode(null, new AffinityKey(0, i)), affProc.mapKeyToNode(null, i));
         }
     }
 
@@ -184,6 +188,17 @@ public class GridBinaryAffinityKeySelfTest extends GridCommonAbstractTest {
             });
 
             assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
+
+            grid(0).compute().affinityRun(null, new AffinityKey(0, i), new IgniteRunnable() {
+                @IgniteInstanceResource
+                private Ignite ignite;
+
+                @Override public void run() {
+                    nodeId.set(ignite.configuration().getNodeId());
+                }
+            });
+
+            assertEquals(aff.mapKeyToNode(i).id(), nodeId.get());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2b64e7c8/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
index 041238f..7010463 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridDefaultBinaryMappersBinaryMetaDataSelfTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.binary.BinaryBasicIdMapper;
 import org.apache.ignite.binary.BinaryNameMapper;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -149,6 +150,22 @@ public class GridDefaultBinaryMappersBinaryMetaDataSelfTest extends GridCommonAb
             else
                 assert false : meta.typeName();
         }
+
+        grid().cache(null).put(new AffinityKey<>(1, 1), 1);
+
+        metas = binaries().types();
+
+        assertEquals(3, metas.size());
+
+        for (BinaryType meta : metas) {
+            if (AffinityKey.class.getSimpleName().equals(meta.typeName())) {
+                assertEquals("affKey", meta.affinityKeyFieldName());
+
+                return;
+            }
+        }
+
+        fail("Failed to find metadata for AffinityKey");
     }
 
     /**


[42/50] ignite git commit: IGNITE-3332: IGFS: Implemented unlock task for client nodes.

Posted by vo...@apache.org.
IGNITE-3332: IGFS: Implemented unlock task for client nodes.


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

Branch: refs/heads/ignite-3341
Commit: df60f650e06f1b0782dd76277ae6415f3bc708e1
Parents: 06831b1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 13:43:26 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 13:43:26 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        |  17 ++-
 .../igfs/client/IgfsClientAbstractCallable.java |  14 +-
 .../igfs/client/IgfsClientAffinityCallable.java |   5 -
 .../igfs/client/IgfsClientDeleteCallable.java   |   6 +-
 .../igfs/client/IgfsClientExistsCallable.java   |   5 -
 .../igfs/client/IgfsClientInfoCallable.java     |   5 -
 .../client/IgfsClientListFilesCallable.java     |   5 -
 .../client/IgfsClientListPathsCallable.java     |   5 -
 .../igfs/client/IgfsClientMkdirsCallable.java   |   5 +-
 .../igfs/client/IgfsClientRenameCallable.java   |   5 +-
 .../igfs/client/IgfsClientSetTimesCallable.java |   6 +-
 .../igfs/client/IgfsClientSizeCallable.java     |   5 -
 .../igfs/client/IgfsClientSummaryCallable.java  |   5 -
 .../igfs/client/IgfsClientUpdateCallable.java   |   5 +-
 .../meta/IgfsClientMetaFileUnlockCallable.java  | 130 +++++++++++++++++++
 .../meta/IgfsClientMetaIdsForPathCallable.java  |   5 -
 .../meta/IgfsClientMetaInfoForPathCallable.java |   5 -
 17 files changed, 175 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 30c7d10..8f1bb69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
+import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaFileUnlockCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
@@ -251,9 +252,14 @@ public class IgfsMetaManager extends IgfsManager {
      */
     <T> T runClientTask(IgfsClientAbstractCallable<T> task) {
         try {
-            if (!task.isReadOnly() && cfg.isColocateMetadata())
+            if (task.isAffinityBased() && cfg.isColocateMetadata()) {
                 // If task mutates state and co-location is enabled, we route request to primary node.
-                return compute().affinityCall(cfg.getMetaCacheName(), IgfsUtils.ROOT_ID, task);
+                IgniteUuid affKey = task.affinityKey();
+
+                assert affKey != null;
+
+                return compute().affinityCall(cfg.getMetaCacheName(), affKey, task);
+            }
             else
                 // Otherwise we route to any available data node.
                 return metaCompute().call(task);
@@ -673,6 +679,13 @@ public class IgfsMetaManager extends IgfsManager {
     public void unlock(final IgniteUuid fileId, final IgniteUuid lockId, final long modificationTime,
         final boolean updateSpace, final long space, @Nullable final IgfsFileAffinityRange affRange)
         throws IgniteCheckedException {
+        if (isClient()) {
+            runClientTask(new IgfsClientMetaFileUnlockCallable(cfg.getName(), fileId, lockId, modificationTime,
+                updateSpace, space, affRange));
+
+            return;
+        }
+
         validTxState(false);
 
         if (busyLock.enterBusy()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
index c06619f..a41418f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -88,7 +89,18 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
     /**
      * @return {@code True} if task is read-only and does not change metadata.
      */
-    public abstract boolean isReadOnly();
+    @Nullable public IgniteUuid affinityKey() {
+        return null;
+    }
+
+    /**
+     * Whether this callable is affinity-based.
+     *
+     * @return {@code True} if affinity based.
+     */
+    public boolean isAffinityBased() {
+        return affinityKey() != null;
+    }
 
     /** {@inheritDoc} */
     @Override public final void writeBinary(BinaryWriter writer) throws BinaryObjectException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
index 6e2748d..1668f36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
@@ -74,11 +74,6 @@ public class IgfsClientAffinityCallable extends IgfsClientAbstractCallable<Colle
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeLong(start);
         writer.writeLong(len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
index 21f8c46..31c158b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
@@ -22,7 +22,9 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -61,8 +63,8 @@ public class IgfsClientDeleteCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
+    @Nullable @Override public IgniteUuid affinityKey() {
+        return IgfsUtils.ROOT_ID;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
index 536cdc8..04b63d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
@@ -52,11 +52,6 @@ public class IgfsClientExistsCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientExistsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
index 6bab8b8..f97c3c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
@@ -53,11 +53,6 @@ public class IgfsClientInfoCallable extends IgfsClientAbstractCallable<IgfsFile>
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientInfoCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
index deb8836..325e714 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
@@ -55,11 +55,6 @@ public class IgfsClientListFilesCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListFilesCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
index ad6c858..78b4c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
@@ -54,11 +54,6 @@ public class IgfsClientListPathsCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListPathsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
index 6ecb6ea..5c2a6ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -24,6 +24,7 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Map;
@@ -66,8 +67,8 @@ public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
+    @Nullable @Override public IgniteUuid affinityKey() {
+        return IgfsUtils.ROOT_ID;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
index 142af47..126b290 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
@@ -24,6 +24,7 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -64,8 +65,8 @@ public class IgfsClientRenameCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
+    @Nullable @Override public IgniteUuid affinityKey() {
+        return IgfsUtils.ROOT_ID;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
index 8c302bc..1e18690 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
@@ -22,7 +22,9 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -69,8 +71,8 @@ public class IgfsClientSetTimesCallable extends IgfsClientAbstractCallable<Void>
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
+    @Nullable @Override public IgniteUuid affinityKey() {
+        return IgfsUtils.ROOT_ID;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
index 29a7b7a..474a940 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
@@ -53,11 +53,6 @@ public class IgfsClientSizeCallable extends IgfsClientAbstractCallable<Long> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSizeCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
index 1e2795c..7e29029 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
@@ -53,11 +53,6 @@ public class IgfsClientSummaryCallable extends IgfsClientAbstractCallable<IgfsPa
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSummaryCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
index 82d6e8d..d9ffe56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -25,6 +25,7 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Map;
@@ -65,8 +66,8 @@ public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFil
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
+    @Nullable @Override public IgniteUuid affinityKey() {
+        return IgfsUtils.ROOT_ID;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java
new file mode 100644
index 0000000..22e512a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java
@@ -0,0 +1,130 @@
+/*
+ * 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.igfs.client.meta;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.processors.igfs.IgfsContext;
+import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
+import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Callable to unlock file info.
+ */
+public class IgfsClientMetaFileUnlockCallable extends IgfsClientAbstractCallable<Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** File ID. */
+    private IgniteUuid fileId;
+
+    /** Lock ID. */
+    private IgniteUuid lockId;
+
+    /** Modification time. */
+    private long modificationTime;
+
+    /** Whether to update space. */
+    private boolean updateSpace;
+
+    /** Space. */
+    private long space;
+
+    /** Affinity range. */
+    private IgfsFileAffinityRange affRange;
+
+    /**
+     * Default constructor.
+     */
+    public IgfsClientMetaFileUnlockCallable() {
+        // NO-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param igfsName IGFS name.
+     * @param fileId File ID.
+     * @param lockId Lock ID.
+     * @param modificationTime Modification time.
+     */
+    public IgfsClientMetaFileUnlockCallable(@Nullable String igfsName, IgniteUuid fileId, IgniteUuid lockId,
+        long modificationTime, boolean updateSpace, long space, @Nullable IgfsFileAffinityRange affRange) {
+        super(igfsName, null);
+
+        this.fileId = fileId;
+        this.lockId = lockId;
+        this.modificationTime = modificationTime;
+        this.updateSpace = updateSpace;
+        this.space = space;
+        this.affRange = affRange;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Void call0(IgfsContext ctx) throws Exception {
+        ctx.meta().unlock(fileId, lockId, modificationTime, updateSpace, space, affRange);
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteUuid affinityKey() {
+        return fileId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
+        BinaryUtils.writeIgniteUuid(writer, fileId);
+        BinaryUtils.writeIgniteUuid(writer, lockId);
+
+        writer.writeLong(modificationTime);
+
+        if (updateSpace) {
+            writer.writeBoolean(true);
+            writer.writeLong(space);
+            writer.writeObject(affRange);
+        }
+        else
+            writer.writeBoolean(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
+        fileId = BinaryUtils.readIgniteUuid(reader);
+        lockId = BinaryUtils.readIgniteUuid(reader);
+
+        modificationTime = reader.readLong();
+
+        if (reader.readBoolean()) {
+            updateSpace = true;
+
+            space = reader.readLong();
+            affRange = reader.readObject();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsClientMetaFileUnlockCallable.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
index b07a0cb..7b3d142 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
@@ -59,11 +59,6 @@ public class IgfsClientMetaIdsForPathCallable extends IgfsClientAbstractCallable
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaIdsForPathCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df60f650/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
index e7e0164..cb31663 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
@@ -57,11 +57,6 @@ public class IgfsClientMetaInfoForPathCallable extends IgfsClientAbstractCallabl
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaInfoForPathCallable.class, this);
     }


[47/50] ignite git commit: Revert "IGNITE-3332: IGFS: Implemented unlock task for client nodes."

Posted by vo...@apache.org.
Revert "IGNITE-3332: IGFS: Implemented unlock task for client nodes."

This reverts commit df60f650e06f1b0782dd76277ae6415f3bc708e1.


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

Branch: refs/heads/ignite-3341
Commit: 04af846d86d400cb8887d23fea55392e1f625bdf
Parents: bf9de9d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 17:39:31 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 17:39:31 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        |  17 +--
 .../igfs/client/IgfsClientAbstractCallable.java |  14 +-
 .../igfs/client/IgfsClientAffinityCallable.java |   5 +
 .../igfs/client/IgfsClientDeleteCallable.java   |   6 +-
 .../igfs/client/IgfsClientExistsCallable.java   |   5 +
 .../igfs/client/IgfsClientInfoCallable.java     |   5 +
 .../client/IgfsClientListFilesCallable.java     |   5 +
 .../client/IgfsClientListPathsCallable.java     |   5 +
 .../igfs/client/IgfsClientMkdirsCallable.java   |   5 +-
 .../igfs/client/IgfsClientRenameCallable.java   |   5 +-
 .../igfs/client/IgfsClientSetTimesCallable.java |   6 +-
 .../igfs/client/IgfsClientSizeCallable.java     |   5 +
 .../igfs/client/IgfsClientSummaryCallable.java  |   5 +
 .../igfs/client/IgfsClientUpdateCallable.java   |   5 +-
 .../meta/IgfsClientMetaFileUnlockCallable.java  | 130 -------------------
 .../meta/IgfsClientMetaIdsForPathCallable.java  |   5 +
 .../meta/IgfsClientMetaInfoForPathCallable.java |   5 +
 17 files changed, 58 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 8f1bb69..30c7d10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -48,7 +48,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheInternal;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
-import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaFileUnlockCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
@@ -252,14 +251,9 @@ public class IgfsMetaManager extends IgfsManager {
      */
     <T> T runClientTask(IgfsClientAbstractCallable<T> task) {
         try {
-            if (task.isAffinityBased() && cfg.isColocateMetadata()) {
+            if (!task.isReadOnly() && cfg.isColocateMetadata())
                 // If task mutates state and co-location is enabled, we route request to primary node.
-                IgniteUuid affKey = task.affinityKey();
-
-                assert affKey != null;
-
-                return compute().affinityCall(cfg.getMetaCacheName(), affKey, task);
-            }
+                return compute().affinityCall(cfg.getMetaCacheName(), IgfsUtils.ROOT_ID, task);
             else
                 // Otherwise we route to any available data node.
                 return metaCompute().call(task);
@@ -679,13 +673,6 @@ public class IgfsMetaManager extends IgfsManager {
     public void unlock(final IgniteUuid fileId, final IgniteUuid lockId, final long modificationTime,
         final boolean updateSpace, final long space, @Nullable final IgfsFileAffinityRange affRange)
         throws IgniteCheckedException {
-        if (isClient()) {
-            runClientTask(new IgfsClientMetaFileUnlockCallable(cfg.getName(), fileId, lockId, modificationTime,
-                updateSpace, space, affRange));
-
-            return;
-        }
-
         validTxState(false);
 
         if (busyLock.enterBusy()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
index a41418f..c06619f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.lang.IgniteCallable;
-import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -89,18 +88,7 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
     /**
      * @return {@code True} if task is read-only and does not change metadata.
      */
-    @Nullable public IgniteUuid affinityKey() {
-        return null;
-    }
-
-    /**
-     * Whether this callable is affinity-based.
-     *
-     * @return {@code True} if affinity based.
-     */
-    public boolean isAffinityBased() {
-        return affinityKey() != null;
-    }
+    public abstract boolean isReadOnly();
 
     /** {@inheritDoc} */
     @Override public final void writeBinary(BinaryWriter writer) throws BinaryObjectException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
index 1668f36..6e2748d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
@@ -74,6 +74,11 @@ public class IgfsClientAffinityCallable extends IgfsClientAbstractCallable<Colle
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeLong(start);
         writer.writeLong(len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
index 31c158b..21f8c46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
@@ -22,9 +22,7 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
-import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -63,8 +61,8 @@ public class IgfsClientDeleteCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteUuid affinityKey() {
-        return IgfsUtils.ROOT_ID;
+    @Override public boolean isReadOnly() {
+        return false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
index 04b63d8..536cdc8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
@@ -52,6 +52,11 @@ public class IgfsClientExistsCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientExistsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
index f97c3c4..6bab8b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
@@ -53,6 +53,11 @@ public class IgfsClientInfoCallable extends IgfsClientAbstractCallable<IgfsFile>
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientInfoCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
index 325e714..deb8836 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
@@ -55,6 +55,11 @@ public class IgfsClientListFilesCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListFilesCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
index 78b4c84..ad6c858 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
@@ -54,6 +54,11 @@ public class IgfsClientListPathsCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListPathsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
index 5c2a6ca..6ecb6ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -24,7 +24,6 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Map;
@@ -67,8 +66,8 @@ public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteUuid affinityKey() {
-        return IgfsUtils.ROOT_ID;
+    @Override public boolean isReadOnly() {
+        return false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
index 126b290..142af47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
@@ -24,7 +24,6 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -65,8 +64,8 @@ public class IgfsClientRenameCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteUuid affinityKey() {
-        return IgfsUtils.ROOT_ID;
+    @Override public boolean isReadOnly() {
+        return false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
index 1e18690..8c302bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
@@ -22,9 +22,7 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
-import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -71,8 +69,8 @@ public class IgfsClientSetTimesCallable extends IgfsClientAbstractCallable<Void>
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteUuid affinityKey() {
-        return IgfsUtils.ROOT_ID;
+    @Override public boolean isReadOnly() {
+        return false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
index 474a940..29a7b7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
@@ -53,6 +53,11 @@ public class IgfsClientSizeCallable extends IgfsClientAbstractCallable<Long> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSizeCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
index 7e29029..1e2795c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
@@ -53,6 +53,11 @@ public class IgfsClientSummaryCallable extends IgfsClientAbstractCallable<IgfsPa
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSummaryCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
index d9ffe56..82d6e8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -25,7 +25,6 @@ import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.igfs.IgfsContext;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Map;
@@ -66,8 +65,8 @@ public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFil
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteUuid affinityKey() {
-        return IgfsUtils.ROOT_ID;
+    @Override public boolean isReadOnly() {
+        return false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java
deleted file mode 100644
index 22e512a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaFileUnlockCallable.java
+++ /dev/null
@@ -1,130 +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.igfs.client.meta;
-
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.BinaryRawReader;
-import org.apache.ignite.binary.BinaryRawWriter;
-import org.apache.ignite.internal.binary.BinaryUtils;
-import org.apache.ignite.internal.processors.igfs.IgfsContext;
-import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
-import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCallable;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Callable to unlock file info.
- */
-public class IgfsClientMetaFileUnlockCallable extends IgfsClientAbstractCallable<Void> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** File ID. */
-    private IgniteUuid fileId;
-
-    /** Lock ID. */
-    private IgniteUuid lockId;
-
-    /** Modification time. */
-    private long modificationTime;
-
-    /** Whether to update space. */
-    private boolean updateSpace;
-
-    /** Space. */
-    private long space;
-
-    /** Affinity range. */
-    private IgfsFileAffinityRange affRange;
-
-    /**
-     * Default constructor.
-     */
-    public IgfsClientMetaFileUnlockCallable() {
-        // NO-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param igfsName IGFS name.
-     * @param fileId File ID.
-     * @param lockId Lock ID.
-     * @param modificationTime Modification time.
-     */
-    public IgfsClientMetaFileUnlockCallable(@Nullable String igfsName, IgniteUuid fileId, IgniteUuid lockId,
-        long modificationTime, boolean updateSpace, long space, @Nullable IgfsFileAffinityRange affRange) {
-        super(igfsName, null);
-
-        this.fileId = fileId;
-        this.lockId = lockId;
-        this.modificationTime = modificationTime;
-        this.updateSpace = updateSpace;
-        this.space = space;
-        this.affRange = affRange;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Void call0(IgfsContext ctx) throws Exception {
-        ctx.meta().unlock(fileId, lockId, modificationTime, updateSpace, space, affRange);
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteUuid affinityKey() {
-        return fileId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
-        BinaryUtils.writeIgniteUuid(writer, fileId);
-        BinaryUtils.writeIgniteUuid(writer, lockId);
-
-        writer.writeLong(modificationTime);
-
-        if (updateSpace) {
-            writer.writeBoolean(true);
-            writer.writeLong(space);
-            writer.writeObject(affRange);
-        }
-        else
-            writer.writeBoolean(false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readBinary0(BinaryRawReader reader) throws BinaryObjectException {
-        fileId = BinaryUtils.readIgniteUuid(reader);
-        lockId = BinaryUtils.readIgniteUuid(reader);
-
-        modificationTime = reader.readLong();
-
-        if (reader.readBoolean()) {
-            updateSpace = true;
-
-            space = reader.readLong();
-            affRange = reader.readObject();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgfsClientMetaFileUnlockCallable.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
index 7b3d142..b07a0cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
@@ -59,6 +59,11 @@ public class IgfsClientMetaIdsForPathCallable extends IgfsClientAbstractCallable
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaIdsForPathCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/04af846d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
index cb31663..e7e0164 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
@@ -57,6 +57,11 @@ public class IgfsClientMetaInfoForPathCallable extends IgfsClientAbstractCallabl
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isReadOnly() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaInfoForPathCallable.class, this);
     }


[23/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh
new file mode 100644
index 0000000..298c1b4
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh
@@ -0,0 +1,373 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+S3_ROOT=s3://bucket/folder
+S3_DOWNLOADS=$S3_ROOT/test
+S3_SYSTEM=$S3_ROOT/test1
+
+CASSANDRA_DOWNLOAD_URL=http://www-eu.apache.org/dist/cassandra/3.5/apache-cassandra-3.5-bin.tar.gz
+CASSANDRA_TARBALL=apache-cassandra-3.5-bin.tar.gz
+CASSANDRA_UNTAR_DIR=apache-cassandra-3.5
+
+TESTS_PACKAGE_DONLOAD_URL=$S3_DOWNLOADS/ignite-cassandra-tests-1.6.0-SNAPSHOT.zip
+TESTS_PACKAGE_ZIP=ignite-cassandra-tests-1.6.0-SNAPSHOT.zip
+TESTS_PACKAGE_UNZIP_DIR=ignite-cassandra-tests
+
+S3_LOGS_URL=$S3_SYSTEM/logs/c-logs
+S3_LOGS_TRIGGER_URL=$S3_SYSTEM/logs-trigger
+S3_BOOTSTRAP_SUCCESS_URL=$S3_SYSTEM/c-success
+S3_BOOTSTRAP_FAILURE_URL=$S3_SYSTEM/c-failure
+S3_CASSANDRA_NODES_DISCOVERY_URL=$S3_SYSTEM/c-discovery
+S3_CASSANDRA_FIRST_NODE_LOCK_URL=$S3_SYSTEM/c-first-node-lock
+S3_CASSANDRA_NODES_JOIN_LOCK_URL=$S3_SYSTEM/c-join-lock
+
+INSTANCE_REGION=us-west-2
+INSTANCE_NAME_TAG=CASSANDRA-SERVER
+INSTANCE_OWNER_TAG=ignite@apache.org
+INSTANCE_PROJECT_TAG=ignite
+
+terminate()
+{
+    if [[ "$S3_BOOTSTRAP_SUCCESS_URL" != */ ]]; then
+        S3_BOOTSTRAP_SUCCESS_URL=${S3_BOOTSTRAP_SUCCESS_URL}/
+    fi
+
+    if [[ "$S3_BOOTSTRAP_FAILURE_URL" != */ ]]; then
+        S3_BOOTSTRAP_FAILURE_URL=${S3_BOOTSTRAP_FAILURE_URL}/
+    fi
+
+    host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+    msg=$host_name
+
+    if [ -n "$1" ]; then
+        echo "[ERROR] $1"
+        echo "[ERROR]-----------------------------------------------------"
+        echo "[ERROR] Cassandra node bootstrap failed"
+        echo "[ERROR]-----------------------------------------------------"
+        msg=$1
+        reportFolder=${S3_BOOTSTRAP_FAILURE_URL}${host_name}
+        reportFile=$reportFolder/__error__
+    else
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Cassandra node bootstrap successfully completed"
+        echo "[INFO]-----------------------------------------------------"
+        reportFolder=${S3_BOOTSTRAP_SUCCESS_URL}${host_name}
+        reportFile=$reportFolder/__success__
+    fi
+
+    echo $msg > /opt/bootstrap-result
+
+    aws s3 rm --recursive $reportFolder
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed drop report folder: $reportFolder"
+    fi
+
+    aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+    fi
+
+    rm -f /opt/bootstrap-result
+
+    if [ -n "$1" ]; then
+        exit 1
+    fi
+
+    exit 0
+}
+
+tagInstance()
+{
+    export EC2_HOME=/opt/aws/apitools/ec2
+    export JAVA_HOME=/opt/jdk1.8.0_77
+    export PATH=$JAVA_HOME/bin:$EC2_HOME/bin:$PATH
+
+    INSTANCE_ID=$(curl http://169.254.169.254/latest/meta-data/instance-id)
+    if [ $? -ne 0 ]; then
+        terminate "Failed to get instance metadata to tag it"
+    fi
+
+    if [ -n "$INSTANCE_NAME_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag Name=${INSTANCE_NAME_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: Name=${INSTANCE_NAME_TAG}"
+        fi
+    fi
+
+    if [ -n "$INSTANCE_OWNER_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag owner=${INSTANCE_OWNER_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: owner=${INSTANCE_OWNER_TAG}"
+        fi
+    fi
+
+    if [ -n "$INSTANCE_PROJECT_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag project=${INSTANCE_PROJECT_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: project=${INSTANCE_PROJECT_TAG}"
+        fi
+    fi
+}
+
+downloadPackage()
+{
+    echo "[INFO] Downloading $3 package from $1 into $2"
+
+    if [[ "$1" == s3* ]]; then
+        aws s3 cp $1 $2
+
+        if [ $? -ne 0 ]; then
+            echo "[WARN] Failed to download $3 package from first attempt"
+            rm -Rf $2
+            sleep 10s
+
+            echo "[INFO] Trying second attempt to download $3 package"
+            aws s3 cp $1 $2
+
+            if [ $? -ne 0 ]; then
+                echo "[WARN] Failed to download $3 package from second attempt"
+                rm -Rf $2
+                sleep 10s
+
+                echo "[INFO] Trying third attempt to download $3 package"
+                aws s3 cp $1 $2
+
+                if [ $? -ne 0 ]; then
+                    terminate "All three attempts to download $3 package from $1 are failed"
+                fi
+            fi
+        fi
+    else
+        curl "$1" -o "$2"
+
+        if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+            echo "[WARN] Failed to download $3 package from first attempt"
+            rm -Rf $2
+            sleep 10s
+
+            echo "[INFO] Trying second attempt to download $3 package"
+            curl "$1" -o "$2"
+
+            if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+                echo "[WARN] Failed to download $3 package from second attempt"
+                rm -Rf $2
+                sleep 10s
+
+                echo "[INFO] Trying third attempt to download $3 package"
+                curl "$1" -o "$2"
+
+                if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+                    terminate "All three attempts to download $3 package from $1 are failed"
+                fi
+            fi
+        fi
+    fi
+
+    echo "[INFO] $3 package successfully downloaded from $1 into $2"
+}
+
+if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+    S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Cassandra node"
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Cassandra download URL: $CASSANDRA_DOWNLOAD_URL"
+echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+echo "[INFO] Logs URL: $S3_LOGS_URL"
+echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER_URL"
+echo "[INFO] Cassandra nodes discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+echo "[INFO] Cassandra first node lock URL: $S3_CASSANDRA_FIRST_NODE_LOCK_URL"
+echo "[INFO] Cassandra nodes join lock URL: $S3_CASSANDRA_NODES_JOIN_LOCK_URL"
+echo "[INFO] Bootsrap success URL: $S3_BOOTSTRAP_SUCCESS_URL"
+echo "[INFO] Bootsrap failure URL: $S3_BOOTSTRAP_FAILURE_URL"
+echo "[INFO]-----------------------------------------------------------------"
+
+echo "[INFO] Installing 'wget' package"
+yum -y install wget
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'wget' package"
+fi
+
+echo "[INFO] Installing 'net-tools' package"
+yum -y install net-tools
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'net-tools' package"
+fi
+
+echo "[INFO] Installing 'python' package"
+yum -y install python
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'python' package"
+fi
+
+echo "[INFO] Installing 'unzip' package"
+yum -y install unzip
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'unzip' package"
+fi
+
+rm -Rf /opt/jdk1.8.0_77 /opt/jdk-8u77-linux-x64.tar.gz
+
+echo "[INFO] Downloading 'jdk-8u77'"
+wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz" -O /opt/jdk-8u77-linux-x64.tar.gz
+if [ $? -ne 0 ]; then
+    terminate "Failed to download 'jdk-8u77'"
+fi
+
+echo "[INFO] Unzipping 'jdk-8u77'"
+tar -xvzf /opt/jdk-8u77-linux-x64.tar.gz -C /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to untar 'jdk-8u77'"
+fi
+
+rm -Rf /opt/jdk-8u77-linux-x64.tar.gz
+
+downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+echo "[INFO] Installing 'pip'"
+python /opt/get-pip.py
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'pip'"
+fi
+
+echo "[INFO] Installing 'awscli'"
+pip install --upgrade awscli
+if [ $? -ne 0 ]; then
+    echo "[ERROR] Failed to install 'awscli' using pip"
+    echo "[INFO] Trying to install awscli using zip archive"
+    echo "[INFO] Downloading awscli zip"
+
+    downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+    echo "[INFO] Unzipping awscli zip"
+    unzip /opt/awscli-bundle.zip -d /opt
+    if [ $? -ne 0 ]; then
+        terminate "Failed to unzip awscli zip"
+    fi
+
+    rm -Rf /opt/awscli-bundle.zip
+
+    echo "[INFO] Installing awscli"
+    /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+    if [ $? -ne 0 ]; then
+        terminate "Failed to install awscli"
+    fi
+
+    echo "[INFO] Successfully installed awscli from zip archive"
+fi
+
+tagInstance
+
+echo "[INFO] Creating 'cassandra' group"
+exists=$(cat /etc/group | grep cassandra)
+if [ -z "$exists" ]; then
+    groupadd cassandra
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create 'cassandra' group"
+    fi
+fi
+
+echo "[INFO] Creating 'cassandra' user"
+exists=$(cat /etc/passwd | grep cassandra)
+if [ -z "$exists" ]; then
+    useradd -g cassandra cassandra
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create 'cassandra' user"
+    fi
+fi
+
+rm -Rf /storage/cassandra /opt/cassandra /opt/$CASSANDRA_TARBALL
+
+echo "[INFO] Creating '/storage/cassandra' storage"
+mkdir -p /storage/cassandra
+chown -R cassandra:cassandra /storage/cassandra
+if [ $? -ne 0 ]; then
+    terminate "Failed to setup Cassandra storage dir: /storage/cassandra"
+fi
+
+downloadPackage "$CASSANDRA_DOWNLOAD_URL" "/opt/$CASSANDRA_TARBALL" "Cassandra"
+
+echo "[INFO] Unzipping Cassandra package"
+tar -xvzf /opt/$CASSANDRA_TARBALL -C /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to untar Cassandra package"
+fi
+
+rm -f /opt/$CASSANDRA_TARBALL /opt/cassandra
+mv /opt/$CASSANDRA_UNTAR_DIR /opt/cassandra
+chown -R cassandra:cassandra /opt/cassandra
+
+downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/$TESTS_PACKAGE_ZIP" "Tests"
+
+unzip /opt/$TESTS_PACKAGE_ZIP -d /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to unzip tests package: $TESTS_PACKAGE_DONLOAD_URL"
+fi
+
+chown -R cassandra:cassandra /opt/$TESTS_PACKAGE_UNZIP_DIR
+find /opt/$TESTS_PACKAGE_UNZIP_DIR -type f -name "*.sh" -exec chmod ug+x {} \;
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/cassandra/cassandra-env.sh" ]; then
+    terminate "There are no cassandra-env.sh in tests package"
+fi
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/cassandra/cassandra-start.sh" ]; then
+    terminate "There are no cassandra-start.sh in tests package"
+fi
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/cassandra/cassandra-template.yaml" ]; then
+    terminate "There are no cassandra-start.sh in tests package"
+fi
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/logs-collector.sh" ]; then
+    terminate "There are no logs-collector.sh in tests package"
+fi
+
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/cassandra/cassandra-start.sh /opt
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/cassandra/cassandra-env.sh /opt/cassandra/conf
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/cassandra/cassandra-template.yaml /opt/cassandra/conf
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/logs-collector.sh /opt
+rm -Rf /opt/$TESTS_PACKAGE_UNZIP_DIR
+chown -R cassandra:cassandra /opt/cassandra /opt/cassandra-start.sh /opt/logs-collector.sh
+
+#profile=/home/cassandra/.bash_profile
+profile=/root/.bash_profile
+
+echo "export JAVA_HOME=/opt/jdk1.8.0_77" >> $profile
+echo "export CASSANDRA_HOME=/opt/cassandra" >> $profile
+echo "export PATH=\$JAVA_HOME/bin:\$CASSANDRA_HOME/bin:\$PATH" >> $profile
+echo "export S3_BOOTSTRAP_SUCCESS_URL=$S3_BOOTSTRAP_SUCCESS_URL" >> $profile
+echo "export S3_BOOTSTRAP_FAILURE_URL=$S3_BOOTSTRAP_FAILURE_URL" >> $profile
+echo "export S3_CASSANDRA_NODES_DISCOVERY_URL=$S3_CASSANDRA_NODES_DISCOVERY_URL" >> $profile
+echo "export S3_CASSANDRA_NODES_JOIN_LOCK_URL=$S3_CASSANDRA_NODES_JOIN_LOCK_URL" >> $profile
+echo "export S3_CASSANDRA_FIRST_NODE_LOCK_URL=$S3_CASSANDRA_FIRST_NODE_LOCK_URL" >> $profile
+
+HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+/opt/logs-collector.sh "/opt/cassandra/logs" "$S3_LOGS_URL/$HOST_NAME" "$S3_LOGS_TRIGGER_URL" > /opt/cassandra/logs-collector.log &
+
+cmd="/opt/cassandra-start.sh"
+
+#sudo -u cassandra -g cassandra sh -c "$cmd | tee /opt/cassandra/start.log"
+
+$cmd | tee /opt/cassandra/start.log
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-env.sh b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-env.sh
new file mode 100644
index 0000000..11dfc50
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-env.sh
@@ -0,0 +1,283 @@
+#
+# 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.
+#
+
+calculate_heap_sizes()
+{
+    case "`uname`" in
+        Linux)
+            system_memory_in_mb=`free -m | awk '/:/ {print $2;exit}'`
+            system_cpu_cores=`egrep -c 'processor([[:space:]]+):.*' /proc/cpuinfo`
+        ;;
+        FreeBSD)
+            system_memory_in_bytes=`sysctl hw.physmem | awk '{print $2}'`
+            system_memory_in_mb=`expr $system_memory_in_bytes / 1024 / 1024`
+            system_cpu_cores=`sysctl hw.ncpu | awk '{print $2}'`
+        ;;
+        SunOS)
+            system_memory_in_mb=`prtconf | awk '/Memory size:/ {print $3}'`
+            system_cpu_cores=`psrinfo | wc -l`
+        ;;
+        Darwin)
+            system_memory_in_bytes=`sysctl hw.memsize | awk '{print $2}'`
+            system_memory_in_mb=`expr $system_memory_in_bytes / 1024 / 1024`
+            system_cpu_cores=`sysctl hw.ncpu | awk '{print $2}'`
+        ;;
+        *)
+            # assume reasonable defaults for e.g. a modern desktop or
+            # cheap server
+            system_memory_in_mb="2048"
+            system_cpu_cores="2"
+        ;;
+    esac
+
+    # some systems like the raspberry pi don't report cores, use at least 1
+    if [ "$system_cpu_cores" -lt "1" ]
+    then
+        system_cpu_cores="1"
+    fi
+
+    # set max heap size based on the following
+    # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
+    # calculate 1/2 ram and cap to 1024MB
+    # calculate 1/4 ram and cap to 8192MB
+    # pick the max
+    half_system_memory_in_mb=`expr $system_memory_in_mb / 2`
+    quarter_system_memory_in_mb=`expr $half_system_memory_in_mb / 2`
+    if [ "$half_system_memory_in_mb" -gt "1024" ]
+    then
+        half_system_memory_in_mb="1024"
+    fi
+    if [ "$quarter_system_memory_in_mb" -gt "8192" ]
+    then
+        quarter_system_memory_in_mb="8192"
+    fi
+    if [ "$half_system_memory_in_mb" -gt "$quarter_system_memory_in_mb" ]
+    then
+        max_heap_size_in_mb="$half_system_memory_in_mb"
+    else
+        max_heap_size_in_mb="$quarter_system_memory_in_mb"
+    fi
+    MAX_HEAP_SIZE="${max_heap_size_in_mb}M"
+
+    # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4 * heap size)
+    max_sensible_yg_per_core_in_mb="100"
+    max_sensible_yg_in_mb=`expr $max_sensible_yg_per_core_in_mb "*" $system_cpu_cores`
+
+    desired_yg_in_mb=`expr $max_heap_size_in_mb / 4`
+
+    if [ "$desired_yg_in_mb" -gt "$max_sensible_yg_in_mb" ]
+    then
+        HEAP_NEWSIZE="${max_sensible_yg_in_mb}M"
+    else
+        HEAP_NEWSIZE="${desired_yg_in_mb}M"
+    fi
+}
+
+# Determine the sort of JVM we'll be running on.
+java_ver_output=`"${JAVA:-java}" -version 2>&1`
+jvmver=`echo "$java_ver_output" | grep '[openjdk|java] version' | awk -F'"' 'NR==1 {print $2}'`
+JVM_VERSION=${jvmver%_*}
+JVM_PATCH_VERSION=${jvmver#*_}
+
+if [ "$JVM_VERSION" \< "1.8" ] ; then
+    echo "Cassandra 3.0 and later require Java 8u40 or later."
+    exit 1;
+fi
+
+if [ "$JVM_VERSION" \< "1.8" ] && [ "$JVM_PATCH_VERSION" \< "40" ] ; then
+    echo "Cassandra 3.0 and later require Java 8u40 or later."
+    exit 1;
+fi
+
+jvm=`echo "$java_ver_output" | grep -A 1 'java version' | awk 'NR==2 {print $1}'`
+case "$jvm" in
+    OpenJDK)
+        JVM_VENDOR=OpenJDK
+        # this will be "64-Bit" or "32-Bit"
+        JVM_ARCH=`echo "$java_ver_output" | awk 'NR==3 {print $2}'`
+        ;;
+    "Java(TM)")
+        JVM_VENDOR=Oracle
+        # this will be "64-Bit" or "32-Bit"
+        JVM_ARCH=`echo "$java_ver_output" | awk 'NR==3 {print $3}'`
+        ;;
+    *)
+        # Help fill in other JVM values
+        JVM_VENDOR=other
+        JVM_ARCH=unknown
+        ;;
+esac
+
+# Override these to set the amount of memory to allocate to the JVM at
+# start-up. For production use you may wish to adjust this for your
+# environment. MAX_HEAP_SIZE is the total amount of memory dedicated
+# to the Java heap. HEAP_NEWSIZE refers to the size of the young
+# generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
+# or not (if you set one, set the other).
+#
+# The main trade-off for the young generation is that the larger it
+# is, the longer GC pause times will be. The shorter it is, the more
+# expensive GC will be (usually).
+#
+# The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent pause
+# times. If in doubt, and if you do not particularly want to tweak, go with
+# 100 MB per physical CPU core.
+
+#MAX_HEAP_SIZE="4G"
+#HEAP_NEWSIZE="800M"
+
+# Set this to control the amount of arenas per-thread in glibc
+#export MALLOC_ARENA_MAX=4
+
+# only calculate the size if it's not set manually
+if [ "x$MAX_HEAP_SIZE" = "x" ] && [ "x$HEAP_NEWSIZE" = "x" ]; then
+    calculate_heap_sizes
+else
+    if [ "x$MAX_HEAP_SIZE" = "x" ] ||  [ "x$HEAP_NEWSIZE" = "x" ]; then
+        echo "please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs (see cassandra-env.sh)"
+        exit 1
+    fi
+fi
+
+if [ "x$MALLOC_ARENA_MAX" = "x" ] ; then
+    export MALLOC_ARENA_MAX=4
+fi
+
+#GC log path has to be defined here because it needs to access CASSANDRA_HOME
+JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
+
+# Here we create the arguments that will get passed to the jvm when
+# starting cassandra.
+
+# Read user-defined JVM options from jvm.options file
+JVM_OPTS_FILE=$CASSANDRA_CONF/jvm.options
+for opt in `grep "^-" $JVM_OPTS_FILE`
+do
+  JVM_OPTS="$JVM_OPTS $opt"
+done
+
+# Check what parameters were defined on jvm.options file to avoid conflicts
+echo $JVM_OPTS | grep -q Xmn
+DEFINED_XMN=$?
+echo $JVM_OPTS | grep -q Xmx
+DEFINED_XMX=$?
+echo $JVM_OPTS | grep -q Xms
+DEFINED_XMS=$?
+echo $JVM_OPTS | grep -q UseConcMarkSweepGC
+USING_CMS=$?
+
+# We only set -Xms and -Xmx if they were not defined on jvm.options file
+# If defined, both Xmx and Xms should be defined together.
+if [ $DEFINED_XMX -ne 0 ] && [ $DEFINED_XMS -ne 0 ]; then
+     JVM_OPTS="$JVM_OPTS -Xms${MAX_HEAP_SIZE}"
+     JVM_OPTS="$JVM_OPTS -Xmx${MAX_HEAP_SIZE}"
+elif [ $DEFINED_XMX -ne 0 ] || [ $DEFINED_XMS -ne 0 ]; then
+     echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
+     exit 1
+fi
+
+# We only set -Xmn flag if it was not defined in jvm.options file
+# and if the CMS GC is being used
+# If defined, both Xmn and Xmx should be defined together.
+if [ $DEFINED_XMN -eq 0 ] && [ $DEFINED_XMX -ne 0 ]; then
+    echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
+    exit 1
+elif [ $DEFINED_XMN -ne 0 ] && [ $USING_CMS -eq 0 ]; then
+    JVM_OPTS="$JVM_OPTS -Xmn${HEAP_NEWSIZE}"
+fi
+
+if [ "$JVM_ARCH" = "64-Bit" ] && [ $USING_CMS -eq 0 ]; then
+    JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark"
+fi
+
+# provides hints to the JIT compiler
+JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler"
+
+# add the jamm javaagent
+JVM_OPTS="$JVM_OPTS -javaagent:$CASSANDRA_HOME/lib/jamm-0.3.0.jar"
+
+# set jvm HeapDumpPath with CASSANDRA_HEAPDUMP_DIR
+if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" ]; then
+    JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
+fi
+
+# jmx: metrics and administration interface
+#
+# add this if you're having trouble connecting:
+# JVM_OPTS="$JVM_OPTS -Djava.rmi.server.hostname=<public name>"
+#
+# see
+# https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
+# for more on configuring JMX through firewalls, etc. (Short version:
+# get it working with no firewall first.)
+#
+# Cassandra ships with JMX accessible *only* from localhost.  
+# To enable remote JMX connections, uncomment lines below
+# with authentication and/or ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity 
+#
+if [ "x$LOCAL_JMX" = "x" ]; then
+    LOCAL_JMX=yes
+fi
+
+# Specifies the default port over which Cassandra will be available for
+# JMX connections.
+# For security reasons, you should not expose this port to the internet.  Firewall it if needed.
+JMX_PORT="7199"
+
+if [ "$LOCAL_JMX" = "yes" ]; then
+#  JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.local.only=false"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
+  JVM_OPTS="$JVM_OPTS -XX:+UnlockCommercialFeatures"
+  JVM_OPTS="$JVM_OPTS -XX:+FlightRecorder"
+  JVM_OPTS="$JVM_OPTS -XX:FlightRecorderOptions=defaultrecording=true"
+else
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
+  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.password.file=/etc/cassandra/jmxremote.password"
+#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=/path/to/keystore"
+#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword=<keystore-password>"
+#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=/path/to/truststore"
+#  JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword=<truststore-password>"
+#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
+#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.registry.ssl=true"
+#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols=<enabled-protocols>"
+#  JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites=<enabled-cipher-suites>"
+fi
+
+# To use mx4j, an HTML interface for JMX, add mx4j-tools.jar to the lib/
+# directory.
+# See http://wiki.apache.org/cassandra/Operations#Monitoring_with_MX4J
+# By default mx4j listens on 0.0.0.0:8081. Uncomment the following lines
+# to control its listen address and port.
+#MX4J_ADDRESS="-Dmx4jaddress=127.0.0.1"
+#MX4J_PORT="-Dmx4jport=8081"
+
+# Cassandra uses SIGAR to capture OS metrics CASSANDRA-7838
+# for SIGAR we have to set the java.library.path
+# to the location of the native libraries.
+JVM_OPTS="$JVM_OPTS -Djava.library.path=$CASSANDRA_HOME/lib/sigar-bin"
+
+JVM_OPTS="$JVM_OPTS $MX4J_ADDRESS"
+JVM_OPTS="$JVM_OPTS $MX4J_PORT"
+JVM_OPTS="$JVM_OPTS $JVM_EXTRA_OPTS"

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-start.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-start.sh b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-start.sh
new file mode 100644
index 0000000..c73c509
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-start.sh
@@ -0,0 +1,550 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+#profile=/home/cassandra/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+
+terminate()
+{
+    if [[ "$S3_BOOTSTRAP_SUCCESS_URL" != */ ]]; then
+        S3_BOOTSTRAP_SUCCESS_URL=${S3_BOOTSTRAP_SUCCESS_URL}/
+    fi
+
+    if [[ "$S3_BOOTSTRAP_FAILURE_URL" != */ ]]; then
+        S3_BOOTSTRAP_FAILURE_URL=${S3_BOOTSTRAP_FAILURE_URL}/
+    fi
+
+    msg=$HOST_NAME
+
+    if [ -n "$1" ]; then
+        echo "[ERROR] $1"
+        echo "[ERROR]-----------------------------------------------------"
+        echo "[ERROR] Failed to start Cassandra node"
+        echo "[ERROR]-----------------------------------------------------"
+        msg=$1
+        reportFolder=${S3_BOOTSTRAP_FAILURE_URL}${HOST_NAME}
+        reportFile=$reportFolder/__error__
+    else
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Cassandra node successfully started"
+        echo "[INFO]-----------------------------------------------------"
+        reportFolder=${S3_BOOTSTRAP_SUCCESS_URL}${HOST_NAME}
+        reportFile=$reportFolder/__success__
+    fi
+
+    echo $msg > /opt/cassandra/start_result
+
+    aws s3 rm --recursive $reportFolder
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed drop report folder: $reportFolder"
+    fi
+
+    if [ -d "/opt/cassandra/logs" ]; then
+        aws s3 sync --sse AES256 /opt/cassandra/logs $reportFolder
+        if [ $? -ne 0 ]; then
+            echo "[ERROR] Failed to export Cassandra logs to: $reportFolder"
+        fi
+    fi
+
+    aws s3 cp --sse AES256 /opt/cassandra/start_result $reportFile
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to export node start result to: $reportFile"
+    fi
+
+    rm -f /opt/cassandra/start_result /opt/cassandra/join-lock /opt/cassandra/remote-join-lock
+
+    if [ -n "$1" ]; then
+        exit 1
+    fi
+
+    exit 0
+}
+
+registerNode()
+{
+    echo "[INFO] Registering Cassandra node seed: ${S3_CASSANDRA_NODES_DISCOVERY_URL}$HOST_NAME"
+
+    aws s3 cp --sse AES256 /opt/cassandra/join-lock ${S3_CASSANDRA_NODES_DISCOVERY_URL}$HOST_NAME
+    if [ $? -ne 0 ]; then
+        terminate "Failed to register Cassandra seed info in: ${S3_CASSANDRA_NODES_DISCOVERY_URL}$HOST_NAME"
+    fi
+
+    echo "[INFO] Cassandra node seed successfully registered"
+}
+
+unregisterNode()
+{
+    echo "[INFO] Removing Cassandra node registration from: ${S3_CASSANDRA_NODES_DISCOVERY_URL}$HOST_NAME"
+    aws s3 rm ${S3_CASSANDRA_NODES_DISCOVERY_URL}$HOST_NAME
+    echo "[INFO] Cassandra node registration removed"
+}
+
+cleanupMetadata()
+{
+    echo "[INFO] Running cleanup"
+    aws s3 rm $S3_CASSANDRA_NODES_JOIN_LOCK_URL
+    aws s3 rm --recursive $S3_CASSANDRA_NODES_DISCOVERY_URL
+    aws s3 rm --recursive $S3_BOOTSTRAP_SUCCESS_URL
+    aws s3 rm --recursive $S3_BOOTSTRAP_FAILURE_URL
+    echo "[INFO] Cleanup completed"
+}
+
+setupCassandraSeeds()
+{
+    echo "[INFO] Setting up Cassandra seeds"
+
+    if [ "$FIRST_NODE" == "true" ]; then
+        CASSANDRA_SEEDS=$(hostname -f | tr '[:upper:]' '[:lower:]')
+        echo "[INFO] Using host address as a seed for the first Cassandra node: $CASSANDRA_SEEDS"
+        aws s3 rm --recursive ${S3_CASSANDRA_NODES_DISCOVERY_URL::-1}
+        if [ $? -ne 0 ]; then
+            terminate "Failed to clean Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+        fi
+
+        cat /opt/cassandra/conf/cassandra-template.yaml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CASSANDRA_SEEDS/g" > /opt/cassandra/conf/cassandra.yaml
+
+        return 0
+    fi
+
+    echo "[INFO] Looking for Cassandra seeds in: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+
+    startTime=$(date +%s)
+
+    while true; do
+        seeds=$(aws s3 ls $S3_CASSANDRA_NODES_DISCOVERY_URL | grep -v PRE | sed -r "s/^.* //g")
+        if [ -n "$seeds" ]; then
+            seeds=($seeds)
+            length=${#seeds[@]}
+
+            if [ $length -lt 4 ]; then
+                seed1=${seeds[0]}
+                seed2=${seeds[1]}
+                seed3=${seeds[2]}
+            else
+                pos1=$(($RANDOM%$length))
+                pos2=$(($RANDOM%$length))
+                pos3=$(($RANDOM%$length))
+                seed1=${seeds[${pos1}]}
+                seed2=${seeds[${pos2}]}
+                seed3=${seeds[${pos3}]}
+            fi
+
+            CASSANDRA_SEEDS=$seed1
+            CASSANDRA_SEED=$seed1
+
+            if [ "$seed2" != "$seed1" ] && [ -n "$seed2" ]; then
+                CASSANDRA_SEEDS="$CASSANDRA_SEEDS,$seed2"
+            fi
+
+            if [ "$seed3" != "$seed2" ] && [ "$seed3" != "$seed1" ] && [ -n "$seed3" ]; then
+                CASSANDRA_SEEDS="$CASSANDRA_SEEDS,$seed3"
+            fi
+
+            echo "[INFO] Using Cassandra seeds: $CASSANDRA_SEEDS"
+
+            cat /opt/cassandra/conf/cassandra-template.yaml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CASSANDRA_SEEDS/g" > /opt/cassandra/conf/cassandra.yaml
+
+            return 0
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but first Cassandra node is still not up and running"
+        fi
+
+        echo "[INFO] Waiting for the first Cassandra node to start and publish its seed, time passed ${duration}min"
+
+        sleep 1m
+    done
+}
+
+tryToGetFirstNodeLock()
+{
+    echo "[INFO] Trying to get first node lock"
+
+    checkFirstNodeLockExist
+    if [ $? -ne 0 ]; then
+        return 1
+    fi
+
+    createFirstNodeLock
+
+    sleep 5s
+
+    rm -Rf /opt/cassandra/first-node-lock
+
+    aws s3 cp $S3_CASSANDRA_FIRST_NODE_LOCK_URL /opt/cassandra/first-node-lock
+    if [ $? -ne 0 ]; then
+        echo "[WARN] Failed to check just created first node lock"
+        return 1
+    fi
+
+    first_host=$(cat /opt/cassandra/first-node-lock)
+
+    rm -f /opt/cassandra/first-node-lock
+
+    if [ "$first_host" != "$HOST_NAME" ]; then
+        echo "[INFO] Node $first_host has discarded previously created first node lock"
+        return 1
+    fi
+
+    echo "[INFO] Congratulations, got first node lock"
+
+    return 0
+}
+
+checkFirstNodeLockExist()
+{
+    echo "[INFO] Checking for the first node lock"
+
+    lockExists=$(aws s3 ls $S3_CASSANDRA_FIRST_NODE_LOCK_URL)
+    if [ -n "$lockExists" ]; then
+        echo "[INFO] First node lock already exists"
+        return 1
+    fi
+
+    echo "[INFO] First node lock doesn't exist"
+
+    return 0
+}
+
+createFirstNodeLock()
+{
+    aws s3 cp --sse AES256 /opt/cassandra/join-lock $S3_CASSANDRA_FIRST_NODE_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create first node lock"
+    fi
+    echo "[INFO] Created first node lock"
+}
+
+removeFirstNodeLock()
+{
+    aws s3 rm $S3_CASSANDRA_FIRST_NODE_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to remove first node lock"
+    fi
+    echo "[INFO] Removed first node lock"
+}
+
+tryToGetClusterJoinLock()
+{
+    echo "[INFO] Trying to get cluster join lock"
+
+    checkClusterJoinLockExist
+    if [ $? -ne 0 ]; then
+        return 1
+    fi
+
+    createClusterJoinLock
+
+    sleep 5s
+
+    rm -Rf /opt/cassandra/remote-join-lock
+
+    aws s3 cp $S3_CASSANDRA_NODES_JOIN_LOCK_URL /opt/cassandra/remote-join-lock
+    if [ $? -ne 0 ]; then
+        echo "[WARN] Failed to check just created cluster join lock"
+        return 1
+    fi
+
+    join_host=$(cat /opt/cassandra/remote-join-lock)
+
+    if [ "$join_host" != "$HOST_NAME" ]; then
+        echo "[INFO] Node $first_host has discarded previously created cluster join lock"
+        return 1
+    fi
+
+    echo "[INFO] Congratulations, got cluster join lock"
+
+    return 0
+}
+
+checkClusterJoinLockExist()
+{
+    echo "[INFO] Checking for the cluster join lock"
+
+    lockExists=$(aws s3 ls $S3_CASSANDRA_NODES_JOIN_LOCK_URL)
+    if [ -n "$lockExists" ]; then
+        echo "[INFO] Cluster join lock already exists"
+        return 1
+    fi
+
+    status=$(/opt/cassandra/bin/nodetool -h $CASSANDRA_SEED status)
+    leaving=$(echo $status | grep UL)
+    moving=$(echo $status | grep UM)
+    joining=$(echo $status | grep UJ)
+
+    if [ -n "$leaving" ] || [ -n "$moving" ] || [ -n "$joining" ]; then
+        echo "[INFO] Cluster join lock doesn't exist in S3, but some node still trying to join Cassandra cluster"
+        return 1
+    fi
+
+    echo "[INFO] Cluster join lock doesn't exist"
+
+    return 0
+}
+
+createClusterJoinLock()
+{
+    aws s3 cp --sse AES256 /opt/cassandra/join-lock $S3_CASSANDRA_NODES_JOIN_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create cluster join lock"
+    fi
+    echo "[INFO] Created cluster join lock"
+}
+
+removeClusterJoinLock()
+{
+    aws s3 rm $S3_CASSANDRA_NODES_JOIN_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to remove cluster join lock"
+    fi
+    echo "[INFO] Removed cluster join lock"
+}
+
+waitToJoinCassandraCluster()
+{
+    echo "[INFO] Waiting to join Cassandra cluster"
+
+    while true; do
+        tryToGetClusterJoinLock
+
+        if [ $? -ne 0 ]; then
+            echo "[INFO] Another node is trying to join cluster. Waiting for extra 1min."
+            sleep 1m
+        else
+            echo "[INFO]-------------------------------------------------------------"
+            echo "[INFO] Congratulations, got lock to join Cassandra cluster"
+            echo "[INFO]-------------------------------------------------------------"
+            break
+        fi
+    done
+}
+
+waitFirstCassandraNodeRegistered()
+{
+    echo "[INFO] Waiting for the first Cassandra node to register"
+
+    startTime=$(date +%s)
+
+    while true; do
+        first_host=
+
+        exists=$(aws s3 ls $S3_CASSANDRA_FIRST_NODE_LOCK_URL)
+        if [ -n "$exists" ]; then
+            rm -Rf /opt/cassandra/first-node-lock
+
+            aws s3 cp $S3_CASSANDRA_FIRST_NODE_LOCK_URL /opt/cassandra/first-node-lock
+            if [ $? -ne 0 ]; then
+                terminate "Failed to check existing first node lock"
+            fi
+
+            first_host=$(cat /opt/cassandra/first-node-lock)
+
+            rm -Rf /opt/cassandra/first-node-lock
+        fi
+
+        if [ -n "$first_host" ]; then
+            exists=$(aws s3 ls ${S3_CASSANDRA_NODES_DISCOVERY_URL}${first_host})
+            if [ -n "$exists" ]; then
+                break
+            fi
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but first Cassandra node is still not up and running"
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] First Cassandra node registered"
+}
+
+startCassandra()
+{
+    echo "[INFO]-------------------------------------------------------------"
+    echo "[INFO] Trying attempt $START_ATTEMPT to start Cassandra daemon"
+    echo "[INFO]-------------------------------------------------------------"
+    echo ""
+
+    setupCassandraSeeds
+
+    if [ "$FIRST_NODE" != "true" ]; then
+        waitToJoinCassandraCluster
+    fi
+
+    proc=$(ps -ef | grep java | grep "org.apache.cassandra.service.CassandraDaemon")
+    proc=($proc)
+
+    if [ -n "${proc[1]}" ]; then
+        echo "[INFO] Terminating existing Cassandra process ${proc[1]}"
+        kill -9 ${proc[1]}
+    fi
+
+    echo "[INFO] Starting Cassandra"
+    rm -Rf /opt/cassandra/logs/* /storage/cassandra/*
+    /opt/cassandra/bin/cassandra -R &
+
+    echo "[INFO] Cassandra job id: $!"
+
+    sleep 1m
+
+    START_ATTEMPT=$(( $START_ATTEMPT+1 ))
+}
+
+# Time (in minutes) to wait for the Cassandra node up and running and register it in S3
+NODE_STARTUP_TIME=10
+
+# Number of attempts to start (not first) Cassandra daemon
+NODE_START_ATTEMPTS=3
+
+HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+echo $HOST_NAME > /opt/cassandra/join-lock
+
+START_ATTEMPT=0
+
+FIRST_NODE="false"
+
+unregisterNode
+
+tryToGetFirstNodeLock
+
+if [ $? -eq 0 ]; then
+    FIRST_NODE="true"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE" == "true" ]; then
+    echo "[INFO] Starting first Cassandra node"
+else
+    echo "[INFO] Starting Cassandra node"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Cassandra nodes discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+echo "[INFO] Cassandra first node lock URL: $S3_CASSANDRA_FIRST_NODE_LOCK_URL"
+echo "[INFO] Cassandra nodes join lock URL: $S3_CASSANDRA_NODES_JOIN_LOCK_URL"
+echo "[INFO] Start success URL: $S3_BOOTSTRAP_SUCCESS_URL"
+echo "[INFO] Start failure URL: $S3_BOOTSTRAP_FAILURE_URL"
+echo "[INFO] CASSANDRA_HOME: $CASSANDRA_HOME"
+echo "[INFO] JAVA_HOME: $JAVA_HOME"
+echo "[INFO] PATH: $PATH"
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ -z "$S3_CASSANDRA_NODES_DISCOVERY_URL" ]; then
+    terminate "S3 discovery URL doesn't specified"
+fi
+
+if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+    S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+fi
+
+if [ "$FIRST_NODE" != "true" ]; then
+    waitFirstCassandraNodeRegistered
+else
+    cleanupMetadata
+fi
+
+startCassandra
+
+startTime=$(date +%s)
+
+while true; do
+    proc=$(ps -ef | grep java | grep "org.apache.cassandra.service.CassandraDaemon")
+
+    /opt/cassandra/bin/nodetool status &> /dev/null
+
+    if [ $? -eq 0 ]; then
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Cassandra daemon successfully started"
+        echo "[INFO]-----------------------------------------------------"
+        echo $proc
+        echo "[INFO]-----------------------------------------------------"
+
+        if [ "$FIRST_NODE" != "true" ]; then
+            removeClusterJoinLock
+        fi
+
+        break
+    fi
+
+    currentTime=$(date +%s)
+    duration=$(( $currentTime-$startTime ))
+    duration=$(( $duration/60 ))
+
+    if [ $duration -gt $NODE_STARTUP_TIME ]; then
+        if [ "$FIRST_NODE" == "true" ]; then
+            removeFirstNodeLock
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but first Cassandra daemon is still not up and running"
+        else
+            removeClusterJoinLock
+
+            if [ $START_ATTEMPT -gt $NODE_START_ATTEMPTS ]; then
+                terminate "${NODE_START_ATTEMPTS} attempts exceed, but Cassandra daemon is still not up and running"
+            fi
+
+            startCassandra
+        fi
+
+        continue
+    fi
+
+    concurrencyError=$(cat /opt/cassandra/logs/system.log | grep "java.lang.UnsupportedOperationException: Other bootstrapping/leaving/moving nodes detected, cannot bootstrap while cassandra.consistent.rangemovement is true")
+
+    if [ -n "$concurrencyError" ] && [ "$FIRST_NODE" != "true" ]; then
+        removeClusterJoinLock
+        echo "[WARN] Failed to concurrently start Cassandra daemon. Sleeping for extra 1min"
+        sleep 1m
+        startCassandra
+        continue
+    fi
+
+    if [ -z "$proc" ]; then
+        if [ "$FIRST_NODE" == "true" ]; then
+            removeFirstNodeLock
+            terminate "Failed to start Cassandra daemon"
+        fi
+
+        removeClusterJoinLock
+        echo "[WARN] Failed to start Cassandra daemon. Sleeping for extra 1min"
+        sleep 1m
+        startCassandra
+        continue
+    fi
+
+    echo "[INFO] Waiting for Cassandra daemon to start, time passed ${duration}min"
+    sleep 30s
+done
+
+registerNode
+
+terminate
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-template.yaml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-template.yaml b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-template.yaml
new file mode 100644
index 0000000..965e34e
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/cassandra/cassandra-template.yaml
@@ -0,0 +1,889 @@
+#
+# 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.
+#
+
+# Cassandra storage config YAML
+
+# NOTE:
+#   See http://wiki.apache.org/cassandra/StorageConfiguration for
+#   full explanations of configuration directives
+# /NOTE
+
+# The name of the cluster. This is mainly used to prevent machines in
+# one logical cluster from joining another.
+cluster_name: 'CassandraIgnite'
+
+# It makes new (non-seed) nodes automatically migrate the right data to themselves.
+# When initializing a fresh cluster with no data, add auto_bootstrap: false
+auto_bootstrap: false
+
+# This defines the number of tokens randomly assigned to this node on the ring
+# The more tokens, relative to other nodes, the larger the proportion of data
+# that this node will store. You probably want all nodes to have the same number
+# of tokens assuming they have equal hardware capability.
+#
+# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility,
+# and will use the initial_token as described below.
+#
+# Specifying initial_token will override this setting on the node's initial start,
+# on subsequent starts, this setting will apply even if initial token is set.
+#
+# If you already have a cluster with 1 token per node, and wish to migrate to 
+# multiple tokens per node, see http://wiki.apache.org/cassandra/Operations
+num_tokens: 256
+
+# initial_token allows you to specify tokens manually.  While you can use # it with
+# vnodes (num_tokens > 1, above) -- in which case you should provide a 
+# comma-separated list -- it's primarily used when adding nodes # to legacy clusters 
+# that do not have vnodes enabled.
+# initial_token:
+
+# See http://wiki.apache.org/cassandra/HintedHandoff
+# May either be "true" or "false" to enable globally, or contain a list
+# of data centers to enable per-datacenter.
+# hinted_handoff_enabled: DC1,DC2
+hinted_handoff_enabled: true
+# this defines the maximum amount of time a dead host will have hints
+# generated.  After it has been dead this long, new hints for it will not be
+# created until it has been seen alive and gone down again.
+max_hint_window_in_ms: 10800000 # 3 hours
+# Maximum throttle in KBs per second, per delivery thread.  This will be
+# reduced proportionally to the number of nodes in the cluster.  (If there
+# are two nodes in the cluster, each delivery thread will use the maximum
+# rate; if there are three, each will throttle to half of the maximum,
+# since we expect two nodes to be delivering hints simultaneously.)
+hinted_handoff_throttle_in_kb: 1024
+# Number of threads with which to deliver hints;
+# Consider increasing this number when you have multi-dc deployments, since
+# cross-dc handoff tends to be slower
+max_hints_delivery_threads: 2
+
+# Maximum throttle in KBs per second, total. This will be
+# reduced proportionally to the number of nodes in the cluster.
+batchlog_replay_throttle_in_kb: 1024
+
+# Authentication backend, implementing IAuthenticator; used to identify users
+# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator,
+# PasswordAuthenticator}.
+#
+# - AllowAllAuthenticator performs no checks - set it to disable authentication.
+# - PasswordAuthenticator relies on username/password pairs to authenticate
+#   users. It keeps usernames and hashed passwords in system_auth.credentials table.
+#   Please increase system_auth keyspace replication factor if you use this authenticator.
+#   If using PasswordAuthenticator, CassandraRoleManager must also be used (see below)
+#authenticator: PasswordAuthenticator
+authenticator: AllowAllAuthenticator
+
+# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions
+# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer,
+# CassandraAuthorizer}.
+#
+# - AllowAllAuthorizer allows any action to any user - set it to disable authorization.
+# - CassandraAuthorizer stores permissions in system_auth.permissions table. Please
+#   increase system_auth keyspace replication factor if you use this authorizer.
+#authorizer: CassandraAuthorizer
+authorizer: AllowAllAuthorizer
+
+# Part of the Authentication & Authorization backend, implementing IRoleManager; used
+# to maintain grants and memberships between roles.
+# Out of the box, Cassandra provides org.apache.cassandra.auth.CassandraRoleManager,
+# which stores role information in the system_auth keyspace. Most functions of the
+# IRoleManager require an authenticated login, so unless the configured IAuthenticator
+# actually implements authentication, most of this functionality will be unavailable.
+#
+# - CassandraRoleManager stores role data in the system_auth keyspace. Please
+#   increase system_auth keyspace replication factor if you use this role manager.
+role_manager: CassandraRoleManager
+
+# Validity period for roles cache (fetching permissions can be an
+# expensive operation depending on the authorizer). Granted roles are cached for
+# authenticated sessions in AuthenticatedUser and after the period specified
+# here, become eligible for (async) reload.
+# Defaults to 2000, set to 0 to disable.
+# Will be disabled automatically for AllowAllAuthenticator.
+roles_validity_in_ms: 2000
+
+# Refresh interval for roles cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If roles_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as roles_validity_in_ms.
+# roles_update_interval_in_ms: 1000
+
+# Validity period for permissions cache (fetching permissions can be an
+# expensive operation depending on the authorizer, CassandraAuthorizer is
+# one example). Defaults to 2000, set to 0 to disable.
+# Will be disabled automatically for AllowAllAuthorizer.
+permissions_validity_in_ms: 2000
+
+# Refresh interval for permissions cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If permissions_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as permissions_validity_in_ms.
+# permissions_update_interval_in_ms: 1000
+
+# The partitioner is responsible for distributing groups of rows (by
+# partition key) across nodes in the cluster.  You should leave this
+# alone for new clusters.  The partitioner can NOT be changed without
+# reloading all data, so when upgrading you should set this to the
+# same partitioner you were already using.
+#
+# Besides Murmur3Partitioner, partitioners included for backwards
+# compatibility include RandomPartitioner, ByteOrderedPartitioner, and
+# OrderPreservingPartitioner.
+#
+partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+
+# Directories where Cassandra should store data on disk.  Cassandra
+# will spread data evenly across them, subject to the granularity of
+# the configured compaction strategy.
+# If not set, the default directory is $CASSANDRA_HOME/data/data.
+data_file_directories:
+     - /storage/cassandra/data
+
+# commit log.  when running on magnetic HDD, this should be a
+# separate spindle than the data directories.
+# If not set, the default directory is $CASSANDRA_HOME/data/commitlog.
+commitlog_directory: /storage/cassandra/commitlog
+
+# policy for data disk failures:
+# die: shut down gossip and client transports and kill the JVM for any fs errors or
+#      single-sstable errors, so the node can be replaced.
+# stop_paranoid: shut down gossip and client transports even for single-sstable errors,
+#                kill the JVM for errors during startup.
+# stop: shut down gossip and client transports, leaving the node effectively dead, but
+#       can still be inspected via JMX, kill the JVM for errors during startup.
+# best_effort: stop using the failed disk and respond to requests based on
+#              remaining available sstables.  This means you WILL see obsolete
+#              data at CL.ONE!
+# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra
+disk_failure_policy: stop
+
+# policy for commit disk failures:
+# die: shut down gossip and Thrift and kill the JVM, so the node can be replaced.
+# stop: shut down gossip and Thrift, leaving the node effectively dead, but
+#       can still be inspected via JMX.
+# stop_commit: shutdown the commit log, letting writes collect but
+#              continuing to service reads, as in pre-2.0.5 Cassandra
+# ignore: ignore fatal errors and let the batches fail
+commit_failure_policy: stop
+
+# Maximum size of the key cache in memory.
+#
+# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+# minimum, sometimes more. The key cache is fairly tiny for the amount of
+# time it saves, so it's worthwhile to use it at large numbers.
+# The row cache saves even more time, but must contain the entire row,
+# so it is extremely space-intensive. It's best to only use the
+# row cache if you have hot rows or static rows.
+#
+# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+#
+# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache.
+key_cache_size_in_mb:
+
+# Duration in seconds after which Cassandra should
+# save the key cache. Caches are saved to saved_caches_directory as
+# specified in this configuration file.
+#
+# Saved caches greatly improve cold-start speeds, and is relatively cheap in
+# terms of I/O for the key cache. Row cache saving is much more expensive and
+# has limited use.
+#
+# Default is 14400 or 4 hours.
+key_cache_save_period: 14400
+
+# Number of keys from the key cache to save
+# Disabled by default, meaning all keys are going to be saved
+# key_cache_keys_to_save: 100
+
+# Row cache implementation class name.
+# Available implementations:
+#   org.apache.cassandra.cache.OHCProvider                Fully off-heap row cache implementation (default).
+#   org.apache.cassandra.cache.SerializingCacheProvider   This is the row cache implementation availabile
+#                                                         in previous releases of Cassandra.
+# row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+
+# Maximum size of the row cache in memory.
+# Please note that OHC cache implementation requires some additional off-heap memory to manage
+# the map structures and some in-flight memory during operations before/after cache entries can be
+# accounted against the cache capacity. This overhead is usually small compared to the whole capacity.
+# Do not specify more memory that the system can afford in the worst usual situation and leave some
+# headroom for OS block level cache. Do never allow your system to swap.
+#
+# Default value is 0, to disable row caching.
+row_cache_size_in_mb: 0
+
+# Duration in seconds after which Cassandra should save the row cache.
+# Caches are saved to saved_caches_directory as specified in this configuration file.
+#
+# Saved caches greatly improve cold-start speeds, and is relatively cheap in
+# terms of I/O for the key cache. Row cache saving is much more expensive and
+# has limited use.
+#
+# Default is 0 to disable saving the row cache.
+row_cache_save_period: 0
+
+# Number of keys from the row cache to save.
+# Specify 0 (which is the default), meaning all keys are going to be saved
+# row_cache_keys_to_save: 100
+
+# Maximum size of the counter cache in memory.
+#
+# Counter cache helps to reduce counter locks' contention for hot counter cells.
+# In case of RF = 1 a counter cache hit will cause Cassandra to skip the read before
+# write entirely. With RF > 1 a counter cache hit will still help to reduce the duration
+# of the lock hold, helping with hot counter cell updates, but will not allow skipping
+# the read entirely. Only the local (clock, count) tuple of a counter cell is kept
+# in memory, not the whole counter, so it's relatively cheap.
+#
+# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+#
+# Default value is empty to make it "auto" (min(2.5% of Heap (in MB), 50MB)). Set to 0 to disable counter cache.
+# NOTE: if you perform counter deletes and rely on low gcgs, you should disable the counter cache.
+counter_cache_size_in_mb:
+
+# Duration in seconds after which Cassandra should
+# save the counter cache (keys only). Caches are saved to saved_caches_directory as
+# specified in this configuration file.
+#
+# Default is 7200 or 2 hours.
+counter_cache_save_period: 7200
+
+# Number of keys from the counter cache to save
+# Disabled by default, meaning all keys are going to be saved
+# counter_cache_keys_to_save: 100
+
+# The off-heap memory allocator.  Affects storage engine metadata as
+# well as caches.  Experiments show that JEMAlloc saves some memory
+# than the native GCC allocator (i.e., JEMalloc is more
+# fragmentation-resistant).
+# 
+# Supported values are: NativeAllocator, JEMallocAllocator
+#
+# If you intend to use JEMallocAllocator you have to install JEMalloc as library and
+# modify cassandra-env.sh as directed in the file.
+#
+# Defaults to NativeAllocator
+# memory_allocator: NativeAllocator
+
+# saved caches
+# If not set, the default directory is $CASSANDRA_HOME/data/saved_caches.
+saved_caches_directory: /storage/cassandra/saved_caches
+
+# commitlog_sync may be either "periodic" or "batch." 
+# 
+# When in batch mode, Cassandra won't ack writes until the commit log
+# has been fsynced to disk.  It will wait
+# commitlog_sync_batch_window_in_ms milliseconds between fsyncs.
+# This window should be kept short because the writer threads will
+# be unable to do extra work while waiting.  (You may need to increase
+# concurrent_writes for the same reason.)
+#
+# commitlog_sync: batch
+# commitlog_sync_batch_window_in_ms: 2
+#
+# the other option is "periodic" where writes may be acked immediately
+# and the CommitLog is simply synced every commitlog_sync_period_in_ms
+# milliseconds. 
+commitlog_sync: periodic
+commitlog_sync_period_in_ms: 10000
+
+# The size of the individual commitlog file segments.  A commitlog
+# segment may be archived, deleted, or recycled once all the data
+# in it (potentially from each columnfamily in the system) has been
+# flushed to sstables.  
+#
+# The default size is 32, which is almost always fine, but if you are
+# archiving commitlog segments (see commitlog_archiving.properties),
+# then you probably want a finer granularity of archiving; 8 or 16 MB
+# is reasonable.
+commitlog_segment_size_in_mb: 32
+
+# Compression to apply to the commit log. If omitted, the commit log
+# will be written uncompressed.  LZ4, Snappy, and Deflate compressors
+# are supported.
+#commitlog_compression:
+#   - class_name: LZ4Compressor
+#     parameters:
+#         -
+
+# any class that implements the SeedProvider interface and has a
+# constructor that takes a Map<String, String> of parameters will do.
+seed_provider:
+    # Addresses of hosts that are deemed contact points. 
+    # Cassandra nodes use this list of hosts to find each other and learn
+    # the topology of the ring.  You must change this if you are running
+    # multiple nodes!
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          # seeds is actually a comma-delimited list of addresses.
+          # Ex: "<ip1>,<ip2>,<ip3>"
+          - seeds: "${CASSANDRA_SEEDS}"
+
+# For workloads with more data than can fit in memory, Cassandra's
+# bottleneck will be reads that need to fetch data from
+# disk. "concurrent_reads" should be set to (16 * number_of_drives) in
+# order to allow the operations to enqueue low enough in the stack
+# that the OS and drives can reorder them. Same applies to
+# "concurrent_counter_writes", since counter writes read the current
+# values before incrementing and writing them back.
+#
+# On the other hand, since writes are almost never IO bound, the ideal
+# number of "concurrent_writes" is dependent on the number of cores in
+# your system; (8 * number_of_cores) is a good rule of thumb.
+concurrent_reads: 32
+concurrent_writes: 32
+concurrent_counter_writes: 32
+
+# Total memory to use for sstable-reading buffers.  Defaults to
+# the smaller of 1/4 of heap or 512MB.
+# file_cache_size_in_mb: 512
+
+# Total permitted memory to use for memtables. Cassandra will stop 
+# accepting writes when the limit is exceeded until a flush completes,
+# and will trigger a flush based on memtable_cleanup_threshold
+# If omitted, Cassandra will set both to 1/4 the size of the heap.
+# memtable_heap_space_in_mb: 2048
+# memtable_offheap_space_in_mb: 2048
+
+# Ratio of occupied non-flushing memtable size to total permitted size
+# that will trigger a flush of the largest memtable.  Lager mct will
+# mean larger flushes and hence less compaction, but also less concurrent
+# flush activity which can make it difficult to keep your disks fed
+# under heavy write load.
+#
+# memtable_cleanup_threshold defaults to 1 / (memtable_flush_writers + 1)
+# memtable_cleanup_threshold: 0.11
+
+# Specify the way Cassandra allocates and manages memtable memory.
+# Options are:
+#   heap_buffers:    on heap nio buffers
+#   offheap_buffers: off heap (direct) nio buffers
+#   offheap_objects: native memory, eliminating nio buffer heap overhead
+memtable_allocation_type: heap_buffers
+
+# Total space to use for commit logs on disk.
+#
+# If space gets above this value, Cassandra will flush every dirty CF
+# in the oldest segment and remove it.  So a small total commitlog space
+# will tend to cause more flush activity on less-active columnfamilies.
+#
+# The default value is 8192.
+# commitlog_total_space_in_mb: 8192
+
+# This sets the amount of memtable flush writer threads.  These will
+# be blocked by disk io, and each one will hold a memtable in memory
+# while blocked. 
+#
+# memtable_flush_writers defaults to the smaller of (number of disks,
+# number of cores), with a minimum of 2 and a maximum of 8.
+# 
+# If your data directories are backed by SSD, you should increase this
+# to the number of cores.
+#memtable_flush_writers: 8
+
+# A fixed memory pool size in MB for for SSTable index summaries. If left
+# empty, this will default to 5% of the heap size. If the memory usage of
+# all index summaries exceeds this limit, SSTables with low read rates will
+# shrink their index summaries in order to meet this limit.  However, this
+# is a best-effort process. In extreme conditions Cassandra may need to use
+# more than this amount of memory.
+index_summary_capacity_in_mb:
+
+# How frequently index summaries should be resampled.  This is done
+# periodically to redistribute memory from the fixed-size pool to sstables
+# proportional their recent read rates.  Setting to -1 will disable this
+# process, leaving existing index summaries at their current sampling level.
+index_summary_resize_interval_in_minutes: 60
+
+# Whether to, when doing sequential writing, fsync() at intervals in
+# order to force the operating system to flush the dirty
+# buffers. Enable this to avoid sudden dirty buffer flushing from
+# impacting read latencies. Almost always a good idea on SSDs; not
+# necessarily on platters.
+trickle_fsync: false
+trickle_fsync_interval_in_kb: 10240
+
+# TCP port, for commands and data
+# For security reasons, you should not expose this port to the internet.  Firewall it if needed.
+storage_port: 7000
+
+# SSL port, for encrypted communication.  Unused unless enabled in
+# encryption_options
+# For security reasons, you should not expose this port to the internet.  Firewall it if needed.
+ssl_storage_port: 7001
+
+# Address or interface to bind to and tell other Cassandra nodes to connect to.
+# You _must_ change this if you want multiple nodes to be able to communicate!
+#
+# Set listen_address OR listen_interface, not both. Interfaces must correspond
+# to a single address, IP aliasing is not supported.
+#
+# Leaving it blank leaves it up to InetAddress.getLocalHost(). This
+# will always do the Right Thing _if_ the node is properly configured
+# (hostname, name resolution, etc), and the Right Thing is to use the
+# address associated with the hostname (it might not be).
+#
+# Setting listen_address to 0.0.0.0 is always wrong.
+#
+# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address
+# you can specify which should be chosen using listen_interface_prefer_ipv6. If false the first ipv4
+# address will be used. If true the first ipv6 address will be used. Defaults to false preferring
+# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6.
+listen_address: 
+# listen_interface: eth0
+# listen_interface_prefer_ipv6: false
+
+# Address to broadcast to other Cassandra nodes
+# Leaving this blank will set it to the same value as listen_address
+# broadcast_address: 1.2.3.4
+
+# Internode authentication backend, implementing IInternodeAuthenticator;
+# used to allow/disallow connections from peer nodes.
+# internode_authenticator: org.apache.cassandra.auth.AllowAllInternodeAuthenticator
+
+# Whether to start the native transport server.
+# Please note that the address on which the native transport is bound is the
+# same as the rpc_address. The port however is different and specified below.
+start_native_transport: true
+# port for the CQL native transport to listen for clients on
+# For security reasons, you should not expose this port to the internet.  Firewall it if needed.
+native_transport_port: 9042
+# The maximum threads for handling requests when the native transport is used.
+# This is similar to rpc_max_threads though the default differs slightly (and
+# there is no native_transport_min_threads, idle threads will always be stopped
+# after 30 seconds).
+# native_transport_max_threads: 128
+#
+# The maximum size of allowed frame. Frame (requests) larger than this will
+# be rejected as invalid. The default is 256MB.
+# native_transport_max_frame_size_in_mb: 256
+
+# The maximum number of concurrent client connections.
+# The default is -1, which means unlimited.
+# native_transport_max_concurrent_connections: -1
+
+# The maximum number of concurrent client connections per source ip.
+# The default is -1, which means unlimited.
+# native_transport_max_concurrent_connections_per_ip: -1
+
+# Whether to start the thrift rpc server.
+start_rpc: true
+
+# The address or interface to bind the Thrift RPC service and native transport
+# server to.
+#
+# Set rpc_address OR rpc_interface, not both. Interfaces must correspond
+# to a single address, IP aliasing is not supported.
+#
+# Leaving rpc_address blank has the same effect as on listen_address
+# (i.e. it will be based on the configured hostname of the node).
+#
+# Note that unlike listen_address, you can specify 0.0.0.0, but you must also
+# set broadcast_rpc_address to a value other than 0.0.0.0.
+#
+# For security reasons, you should not expose this port to the internet.  Firewall it if needed.
+#
+# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address
+# you can specify which should be chosen using rpc_interface_prefer_ipv6. If false the first ipv4
+# address will be used. If true the first ipv6 address will be used. Defaults to false preferring
+# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6.
+rpc_address: 
+# rpc_interface: eth1
+# rpc_interface_prefer_ipv6: false
+
+# port for Thrift to listen for clients on
+rpc_port: 9160
+
+# RPC address to broadcast to drivers and other Cassandra nodes. This cannot
+# be set to 0.0.0.0. If left blank, this will be set to the value of
+# rpc_address. If rpc_address is set to 0.0.0.0, broadcast_rpc_address must
+# be set.
+broadcast_rpc_address: 
+
+# enable or disable keepalive on rpc/native connections
+rpc_keepalive: true
+
+# Cassandra provides two out-of-the-box options for the RPC Server:
+#
+# sync  -> One thread per thrift connection. For a very large number of clients, memory
+#          will be your limiting factor. On a 64 bit JVM, 180KB is the minimum stack size
+#          per thread, and that will correspond to your use of virtual memory (but physical memory
+#          may be limited depending on use of stack space).
+#
+# hsha  -> Stands for "half synchronous, half asynchronous." All thrift clients are handled
+#          asynchronously using a small number of threads that does not vary with the amount
+#          of thrift clients (and thus scales well to many clients). The rpc requests are still
+#          synchronous (one thread per active request). If hsha is selected then it is essential
+#          that rpc_max_threads is changed from the default value of unlimited.
+#
+# The default is sync because on Windows hsha is about 30% slower.  On Linux,
+# sync/hsha performance is about the same, with hsha of course using less memory.
+#
+# Alternatively,  can provide your own RPC server by providing the fully-qualified class name
+# of an o.a.c.t.TServerFactory that can create an instance of it.
+rpc_server_type: sync
+
+# Uncomment rpc_min|max_thread to set request pool size limits.
+#
+# Regardless of your choice of RPC server (see above), the number of maximum requests in the
+# RPC thread pool dictates how many concurrent requests are possible (but if you are using the sync
+# RPC server, it also dictates the number of clients that can be connected at all).
+#
+# The default is unlimited and thus provides no protection against clients overwhelming the server. You are
+# encouraged to set a maximum that makes sense for you in production, but do keep in mind that
+# rpc_max_threads represents the maximum number of client requests this server may execute concurrently.
+#
+# rpc_min_threads: 16
+# rpc_max_threads: 2048
+
+# uncomment to set socket buffer sizes on rpc connections
+# rpc_send_buff_size_in_bytes:
+# rpc_recv_buff_size_in_bytes:
+
+# Uncomment to set socket buffer size for internode communication
+# Note that when setting this, the buffer size is limited by net.core.wmem_max
+# and when not setting it it is defined by net.ipv4.tcp_wmem
+# See:
+# /proc/sys/net/core/wmem_max
+# /proc/sys/net/core/rmem_max
+# /proc/sys/net/ipv4/tcp_wmem
+# /proc/sys/net/ipv4/tcp_wmem
+# and: man tcp
+# internode_send_buff_size_in_bytes:
+# internode_recv_buff_size_in_bytes:
+
+# Frame size for thrift (maximum message length).
+thrift_framed_transport_size_in_mb: 15
+
+# Set to true to have Cassandra create a hard link to each sstable
+# flushed or streamed locally in a backups/ subdirectory of the
+# keyspace data.  Removing these links is the operator's
+# responsibility.
+incremental_backups: false
+
+# Whether or not to take a snapshot before each compaction.  Be
+# careful using this option, since Cassandra won't clean up the
+# snapshots for you.  Mostly useful if you're paranoid when there
+# is a data format change.
+snapshot_before_compaction: false
+
+# Whether or not a snapshot is taken of the data before keyspace truncation
+# or dropping of column families. The STRONGLY advised default of true 
+# should be used to provide data safety. If you set this flag to false, you will
+# lose data on truncation or drop.
+auto_snapshot: true
+
+# When executing a scan, within or across a partition, we need to keep the
+# tombstones seen in memory so we can return them to the coordinator, which
+# will use them to make sure other replicas also know about the deleted rows.
+# With workloads that generate a lot of tombstones, this can cause performance
+# problems and even exaust the server heap.
+# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets)
+# Adjust the thresholds here if you understand the dangers and want to
+# scan more tombstones anyway.  These thresholds may also be adjusted at runtime
+# using the StorageService mbean.
+tombstone_warn_threshold: 1000
+tombstone_failure_threshold: 100000
+
+# Granularity of the collation index of rows within a partition.
+# Increase if your rows are large, or if you have a very large
+# number of rows per partition.  The competing goals are these:
+#   1) a smaller granularity means more index entries are generated
+#      and looking up rows withing the partition by collation column
+#      is faster
+#   2) but, Cassandra will keep the collation index in memory for hot
+#      rows (as part of the key cache), so a larger granularity means
+#      you can cache more hot rows
+column_index_size_in_kb: 64
+
+
+# Log WARN on any batch size exceeding this value. 5kb per batch by default.
+# Caution should be taken on increasing the size of this threshold as it can lead to node instability.
+batch_size_warn_threshold_in_kb: 5
+
+# Fail any batch exceeding this value. 50kb (10x warn threshold) by default.
+batch_size_fail_threshold_in_kb: 50
+
+# Number of simultaneous compactions to allow, NOT including
+# validation "compactions" for anti-entropy repair.  Simultaneous
+# compactions can help preserve read performance in a mixed read/write
+# workload, by mitigating the tendency of small sstables to accumulate
+# during a single long running compactions. The default is usually
+# fine and if you experience problems with compaction running too
+# slowly or too fast, you should look at
+# compaction_throughput_mb_per_sec first.
+#
+# concurrent_compactors defaults to the smaller of (number of disks,
+# number of cores), with a minimum of 2 and a maximum of 8.
+# 
+# If your data directories are backed by SSD, you should increase this
+# to the number of cores.
+#concurrent_compactors: 1
+
+# Throttles compaction to the given total throughput across the entire
+# system. The faster you insert data, the faster you need to compact in
+# order to keep the sstable count down, but in general, setting this to
+# 16 to 32 times the rate you are inserting data is more than sufficient.
+# Setting this to 0 disables throttling. Note that this account for all types
+# of compaction, including validation compaction.
+compaction_throughput_mb_per_sec: 16
+
+# Log a warning when compacting partitions larger than this value
+compaction_large_partition_warning_threshold_mb: 100
+
+# When compacting, the replacement sstable(s) can be opened before they
+# are completely written, and used in place of the prior sstables for
+# any range that has been written. This helps to smoothly transfer reads 
+# between the sstables, reducing page cache churn and keeping hot rows hot
+sstable_preemptive_open_interval_in_mb: 50
+
+# Throttles all outbound streaming file transfers on this node to the
+# given total throughput in Mbps. This is necessary because Cassandra does
+# mostly sequential IO when streaming data during bootstrap or repair, which
+# can lead to saturating the network connection and degrading rpc performance.
+# When unset, the default is 200 Mbps or 25 MB/s.
+# stream_throughput_outbound_megabits_per_sec: 200
+
+# Throttles all streaming file transfer between the datacenters,
+# this setting allows users to throttle inter dc stream throughput in addition
+# to throttling all network stream traffic as configured with
+# stream_throughput_outbound_megabits_per_sec
+# inter_dc_stream_throughput_outbound_megabits_per_sec:
+
+# How long the coordinator should wait for read operations to complete
+read_request_timeout_in_ms: 50000
+# How long the coordinator should wait for seq or index scans to complete
+range_request_timeout_in_ms: 10000
+# How long the coordinator should wait for writes to complete
+write_request_timeout_in_ms: 20000
+# How long the coordinator should wait for counter writes to complete
+counter_write_request_timeout_in_ms: 5000
+# How long a coordinator should continue to retry a CAS operation
+# that contends with other proposals for the same row
+cas_contention_timeout_in_ms: 1000
+# How long the coordinator should wait for truncates to complete
+# (This can be much longer, because unless auto_snapshot is disabled
+# we need to flush first so we can snapshot before removing the data.)
+truncate_request_timeout_in_ms: 60000
+# The default timeout for other, miscellaneous operations
+request_timeout_in_ms: 10000
+
+# Enable operation timeout information exchange between nodes to accurately
+# measure request timeouts.  If disabled, replicas will assume that requests
+# were forwarded to them instantly by the coordinator, which means that
+# under overload conditions we will waste that much extra time processing 
+# already-timed-out requests.
+#
+# Warning: before enabling this property make sure to ntp is installed
+# and the times are synchronized between the nodes.
+cross_node_timeout: false
+
+# Enable socket timeout for streaming operation.
+# When a timeout occurs during streaming, streaming is retried from the start
+# of the current file. This _can_ involve re-streaming an important amount of
+# data, so you should avoid setting the value too low.
+# Default value is 3600000, which means streams timeout after an hour.
+# streaming_socket_timeout_in_ms: 3600000
+
+# phi value that must be reached for a host to be marked down.
+# most users should never need to adjust this.
+# phi_convict_threshold: 8
+
+# endpoint_snitch -- Set this to a class that implements
+# IEndpointSnitch.  The snitch has two functions:
+# - it teaches Cassandra enough about your network topology to route
+#   requests efficiently
+# - it allows Cassandra to spread replicas around your cluster to avoid
+#   correlated failures. It does this by grouping machines into
+#   "datacenters" and "racks."  Cassandra will do its best not to have
+#   more than one replica on the same "rack" (which may not actually
+#   be a physical location)
+#
+# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER,
+# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS
+# ARE PLACED.
+#
+# Out of the box, Cassandra provides
+#  - SimpleSnitch:
+#    Treats Strategy order as proximity. This can improve cache
+#    locality when disabling read repair.  Only appropriate for
+#    single-datacenter deployments.
+#  - GossipingPropertyFileSnitch
+#    This should be your go-to snitch for production use.  The rack
+#    and datacenter for the local node are defined in
+#    cassandra-rackdc.properties and propagated to other nodes via
+#    gossip.  If cassandra-topology.properties exists, it is used as a
+#    fallback, allowing migration from the PropertyFileSnitch.
+#  - PropertyFileSnitch:
+#    Proximity is determined by rack and data center, which are
+#    explicitly configured in cassandra-topology.properties.
+#  - Ec2Snitch:
+#    Appropriate for EC2 deployments in a single Region. Loads Region
+#    and Availability Zone information from the EC2 API. The Region is
+#    treated as the datacenter, and the Availability Zone as the rack.
+#    Only private IPs are used, so this will not work across multiple
+#    Regions.
+#  - Ec2MultiRegionSnitch:
+#    Uses public IPs as broadcast_address to allow cross-region
+#    connectivity.  (Thus, you should set seed addresses to the public
+#    IP as well.) You will need to open the storage_port or
+#    ssl_storage_port on the public IP firewall.  (For intra-Region
+#    traffic, Cassandra will switch to the private IP after
+#    establishing a connection.)
+#  - RackInferringSnitch:
+#    Proximity is determined by rack and data center, which are
+#    assumed to correspond to the 3rd and 2nd octet of each node's IP
+#    address, respectively.  Unless this happens to match your
+#    deployment conventions, this is best used as an example of
+#    writing a custom Snitch class and is provided in that spirit.
+#
+# You can use a custom Snitch by setting this to the full class name
+# of the snitch, which will be assumed to be on your classpath.
+endpoint_snitch: GossipingPropertyFileSnitch
+
+# controls how often to perform the more expensive part of host score
+# calculation
+dynamic_snitch_update_interval_in_ms: 100 
+# controls how often to reset all host scores, allowing a bad host to
+# possibly recover
+dynamic_snitch_reset_interval_in_ms: 600000
+# if set greater than zero and read_repair_chance is < 1.0, this will allow
+# 'pinning' of replicas to hosts in order to increase cache capacity.
+# The badness threshold will control how much worse the pinned host has to be
+# before the dynamic snitch will prefer other replicas over it.  This is
+# expressed as a double which represents a percentage.  Thus, a value of
+# 0.2 means Cassandra would continue to prefer the static snitch values
+# until the pinned host was 20% worse than the fastest.
+dynamic_snitch_badness_threshold: 0.1
+
+# request_scheduler -- Set this to a class that implements
+# RequestScheduler, which will schedule incoming client requests
+# according to the specific policy. This is useful for multi-tenancy
+# with a single Cassandra cluster.
+# NOTE: This is specifically for requests from the client and does
+# not affect inter node communication.
+# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place
+# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of
+# client requests to a node with a separate queue for each
+# request_scheduler_id. The scheduler is further customized by
+# request_scheduler_options as described below.
+request_scheduler: org.apache.cassandra.scheduler.NoScheduler
+
+# Scheduler Options vary based on the type of scheduler
+# NoScheduler - Has no options
+# RoundRobin
+#  - throttle_limit -- The throttle_limit is the number of in-flight
+#                      requests per client.  Requests beyond 
+#                      that limit are queued up until
+#                      running requests can complete.
+#                      The value of 80 here is twice the number of
+#                      concurrent_reads + concurrent_writes.
+#  - default_weight -- default_weight is optional and allows for
+#                      overriding the default which is 1.
+#  - weights -- Weights are optional and will default to 1 or the
+#               overridden default_weight. The weight translates into how
+#               many requests are handled during each turn of the
+#               RoundRobin, based on the scheduler id.
+#
+# request_scheduler_options:
+#    throttle_limit: 80
+#    default_weight: 5
+#    weights:
+#      Keyspace1: 1
+#      Keyspace2: 5
+
+# request_scheduler_id -- An identifier based on which to perform
+# the request scheduling. Currently the only valid option is keyspace.
+# request_scheduler_id: keyspace
+
+# Enable or disable inter-node encryption
+# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that
+# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher
+# suite for authentication, key exchange and encryption of the actual data transfers.
+# Use the DHE/ECDHE ciphers if running in FIPS 140 compliant mode.
+# NOTE: No custom encryption options are enabled at the moment
+# The available internode options are : all, none, dc, rack
+#
+# If set to dc cassandra will encrypt the traffic between the DCs
+# If set to rack cassandra will encrypt the traffic between the racks
+#
+# The passwords used in these options must match the passwords used when generating
+# the keystore and truststore.  For instructions on generating these files, see:
+# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore
+#
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+    # More advanced defaults below:
+    # protocol: TLS
+    # algorithm: SunX509
+    # store_type: JKS
+    # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA]
+    # require_client_auth: false
+
+# enable or disable client/server encryption.
+client_encryption_options:
+    enabled: false
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    # require_client_auth: false
+    # Set trustore and truststore_password if require_client_auth is true
+    # truststore: conf/.truststore
+    # truststore_password: cassandra
+    # More advanced defaults below:
+    # protocol: TLS
+    # algorithm: SunX509
+    # store_type: JKS
+    # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA]
+
+# internode_compression controls whether traffic between nodes is
+# compressed.
+# can be:  all  - all traffic is compressed
+#          dc   - traffic between different datacenters is compressed
+#          none - nothing is compressed.
+internode_compression: all
+
+# Enable or disable tcp_nodelay for inter-dc communication.
+# Disabling it will result in larger (but fewer) network packets being sent,
+# reducing overhead from the TCP protocol itself, at the cost of increasing
+# latency if you block for cross-datacenter responses.
+inter_dc_tcp_nodelay: false
+
+# TTL for different trace types used during logging of the repair process.
+tracetype_query_ttl: 86400
+tracetype_repair_ttl: 604800
+
+# UDFs (user defined functions) are disabled by default.
+# As of Cassandra 2.2, there is no security manager or anything else in place that
+# prevents execution of evil code. CASSANDRA-9402 will fix this issue for Cassandra 3.0.
+# This will inherently be backwards-incompatible with any 2.2 UDF that perform insecure
+# operations such as opening a socket or writing to the filesystem.
+enable_user_defined_functions: false
+
+# The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation.
+# Lowering this value on Windows can provide much tighter latency and better throughput, however
+# some virtualized environments may see a negative performance impact from changing this setting
+# below their system default. The sysinternals 'clockres' tool can confirm your system's default
+# setting.
+windows_timer_interval: 1


[21/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/tests/tests-report.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/tests/tests-report.sh b/modules/cassandra/src/test/bootstrap/aws/tests/tests-report.sh
new file mode 100644
index 0000000..762dc6f
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/tests/tests-report.sh
@@ -0,0 +1,590 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+
+validate()
+{
+    if [ -z "$TESTS_TYPE" ]; then
+        terminate "Tests type 'ignite' or 'cassandra' should be specified"
+    fi
+
+    if [ "$TESTS_TYPE" != "ignite" ] && [ "$TESTS_TYPE" != "cassandra" ]; then
+        terminate "Incorrect tests type specified: $TESTS_TYPE"
+    fi
+
+    if [ -z "$S3_TESTS_SUCCESS_URL" ]; then
+        terminate "Tests success URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_SUCCESS_URL" != */ ]]; then
+        S3_TESTS_SUCCESS_URL=${S3_TESTS_SUCCESS_URL}/
+    fi
+
+    if [ -z "$S3_TESTS_FAILURE_URL" ]; then
+        terminate "Tests failure URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_FAILURE_URL" != */ ]]; then
+        S3_TESTS_FAILURE_URL=${S3_TESTS_FAILURE_URL}/
+    fi
+
+    if [ -z "$S3_TESTS_RUNNING_URL" ]; then
+        terminate "Tests running URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_RUNNING_URL" != */ ]]; then
+        S3_TESTS_RUNNING_URL=${S3_TESTS_RUNNING_URL}/
+    fi
+
+    if [ -z "$S3_TESTS_WAITING_URL" ]; then
+        terminate "Tests waiting URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_WAITING_URL" != */ ]]; then
+        S3_TESTS_WAITING_URL=${S3_TESTS_WAITING_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_SUCCESS_URL" ]; then
+        terminate "Ignite success URL doesn't specified"
+    fi
+
+    if [[ "$S3_IGNITE_SUCCESS_URL" != */ ]]; then
+        S3_IGNITE_SUCCESS_URL=${S3_IGNITE_SUCCESS_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_FAILURE_URL" ]; then
+        terminate "Ignite failure URL doesn't specified"
+    fi
+
+    if [[ "$S3_IGNITE_FAILURE_URL" != */ ]]; then
+        S3_IGNITE_FAILURE_URL=${S3_IGNITE_FAILURE_URL}/
+    fi
+
+    if [ -z "$S3_CASSANDRA_SUCCESS_URL" ]; then
+        terminate "Cassandra success URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_SUCCESS_URL" != */ ]]; then
+        S3_CASSANDRA_SUCCESS_URL=${S3_CASSANDRA_SUCCESS_URL}/
+    fi
+
+    if [ -z "$S3_CASSANDRA_FAILURE_URL" ]; then
+        terminate "Cassandra failure URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_FAILURE_URL" != */ ]]; then
+        S3_CASSANDRA_FAILURE_URL=${S3_CASSANDRA_FAILURE_URL}/
+    fi
+
+    if [ -z "$S3_TEST_NODES_DISCOVERY_URL" ]; then
+        terminate "Tests S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_TEST_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_TEST_NODES_DISCOVERY_URL=${S3_TEST_NODES_DISCOVERY_URL}/
+    fi
+
+    if [ -z "$S3_CASSANDRA_NODES_DISCOVERY_URL" ]; then
+        terminate "Cassandra S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_NODES_DISCOVERY_URL" ]; then
+        terminate "Ignite S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_NODES_DISCOVERY_URL" ]; then
+        terminate "Ignite S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_IGNITE_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_IGNITE_NODES_DISCOVERY_URL=${S3_IGNITE_NODES_DISCOVERY_URL}/
+    fi
+}
+
+reportTestsSummary()
+{
+    echo "[INFO] Preparing tests results summary"
+
+    TESTS_SUMMARY_DIR=/opt/ignite-cassandra-tests/tests-summary
+    SUCCEED_NODES_FILE=$TESTS_SUMMARY_DIR/succeed-nodes
+    SUCCEED_NODES_DIR=$TESTS_SUMMARY_DIR/succeed
+    FAILED_NODES_FILE=$TESTS_SUMMARY_DIR/failed-nodes
+    FAILED_NODES_DIR=$TESTS_SUMMARY_DIR/failed
+    REPORT_FILE=$TESTS_SUMMARY_DIR/report.txt
+
+    rm -Rf $TESTS_SUMMARY_DIR
+    mkdir -p $TESTS_SUMMARY_DIR
+    mkdir -p $SUCCEED_NODES_DIR
+    mkdir -p $FAILED_NODES_DIR
+
+    aws s3 ls $S3_TESTS_SUCCESS_URL | sed -r "s/PRE //g" | sed -r "s/ //g" | sed -r "s/\///g" > $SUCCEED_NODES_FILE
+    aws s3 ls $S3_TESTS_FAILURE_URL | sed -r "s/PRE //g" | sed -r "s/ //g" | sed -r "s/\///g" > $FAILED_NODES_FILE
+
+    succeedCount=$(cat $SUCCEED_NODES_FILE | wc -l)
+    failedCount=$(cat $FAILED_NODES_FILE | wc -l)
+    count=$(( $succeedCount+$failedCount ))
+
+    echo "Test type         : $TESTS_TYPE" > $REPORT_FILE
+    echo "Test nodes count  : $count" >> $REPORT_FILE
+    echo "Test nodes succeed: $succeedCount" >> $REPORT_FILE
+    echo "Test nodes failed : $failedCount" >> $REPORT_FILE
+    echo "----------------------------------------------------------------------------------------------" >> $REPORT_FILE
+
+    if [ $succeedCount -gt 0 ]; then
+        echo "Succeed test nodes |" >> $REPORT_FILE
+        echo "-------------------" >> $REPORT_FILE
+        cat $SUCCEED_NODES_FILE >> $REPORT_FILE
+        echo "----------------------------------------------------------------------------------------------" >> $REPORT_FILE
+
+        aws s3 sync --delete $S3_TESTS_SUCCESS_URL $SUCCEED_NODES_DIR
+        if [ $? -ne 0 ]; then
+            echo "[ERROR] Failed to get succeed tests details"
+        else
+            reportSucceedTestsStatistics "$REPORT_FILE" "$SUCCEED_NODES_DIR"
+        fi
+    fi
+
+    if [ $failedCount -gt 0 ]; then
+        echo "Failed test nodes |" >> $REPORT_FILE
+        echo "------------------" >> $REPORT_FILE
+        cat $FAILED_NODES_FILE >> $REPORT_FILE
+        echo "----------------------------------------------------------------------------------------------" >> $REPORT_FILE
+
+        aws sync --delete $S3_TESTS_FAILURE_URL $FAILED_NODES_DIR
+        if [ $? -ne 0 ]; then
+            echo "[ERROR] Failed to get failed tests details"
+        else
+            reportFailedTestsDetailes "$REPORT_FILE" "$FAILED_NODES_DIR"
+        fi
+    fi
+
+    rm -f $HOME/tests-summary.zip
+
+    pushd $TESTS_SUMMARY_DIR
+
+    zip -r -9 $HOME/tests-summary.zip .
+    code=$?
+
+    rm -Rf $TESTS_SUMMARY_DIR
+
+    popd
+
+    if [ $code -ne 0 ]; then
+        echo "-------------------------------------------------------------------------------------"
+        echo "[ERROR] Failed to create tests summary zip archive $HOME/tests-summary.zip for $TESTS_SUMMARY_DIR"
+        echo "-------------------------------------------------------------------------------------"
+        return 1
+    fi
+
+    aws s3 cp --sse AES256 $HOME/tests-summary.zip $S3_TESTS_SUMMARY_URL
+    if [ $? -ne 0 ]; then
+        echo "-------------------------------------------------------------------------------------"
+        echo "[ERROR] Failed to uploat tests summary archive to: $S3_TESTS_SUMMARY_URL"
+        echo "-------------------------------------------------------------------------------------"
+    else
+        echo "-------------------------------------------------------------------------------------"
+        echo "[INFO] Tests results summary uploaded to: $S3_TESTS_SUMMARY_URL"
+        echo "-------------------------------------------------------------------------------------"
+    fi
+
+    rm -f $HOME/tests-summary.zip
+}
+
+reportSucceedTestsStatistics()
+{
+    writeMsg=0
+    writeErrors=0
+    writeSpeed=0
+    blkWriteMsg=0
+    blkWriteErrors=0
+    blkWriteSpeed=0
+    readMsg=0
+    readErrors=0
+    readSpeed=0
+    blkReadMsg=0
+    blkReadErrors=0
+    blkReadSpeed=0
+
+    writeErrNodes=
+    blkWriteErrNodes=
+    readErrNodes=
+    blkReadErrNodes=
+
+	tmpFile=`mktemp`
+
+    for dir in $2/*
+    do
+        node=$(echo $dir | sed -r "s/^.*\///g")
+        echo "-------------------------------------------------------------------------------------"
+        echo "[INFO] Gathering statistics from $node test node"
+        echo "-------------------------------------------------------------------------------------"
+
+        logFile=$(ls $dir | grep "${TESTS_TYPE}-load-tests.log" | head -1)
+        if [ -z "$logFile" ]; then
+            echo "[WARN] Node $node marked as succeeded, but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file"
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Node $node marked as succeeded," >> $tmpFile
+            echo "but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file" >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            continue
+        fi
+
+        logFile=$dir/$logFile
+        if [ ! -f "$logFile" ]; then
+            echo "[WARN] Node $node marked as succeeded, but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file"
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Node $node marked as succeeded," >> $tmpFile
+            echo "but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file" >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            continue
+        fi
+
+        cnt=$(cat $logFile | grep "^WRITE messages" | sed -r "s/WRITE messages: //g" | xargs)
+        if [ -n "$cnt" ]; then
+            writeMsg=$(( $writeMsg+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] WRITE messages: $cnt"
+            else
+                echo "[WARN] WRITE messages count is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "WRITE messages count is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect WRITE messages count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect WRITE messages count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^WRITE errors" | sed -r "s/WRITE errors: //g" | sed -r "s/,.*//g" | xargs)
+        if [ -n "$cnt" ]; then
+            echo "[INFO] WRITE errors: $cnt"
+            writeErrors=$(( $writeErrors+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                if [ -n "$writeErrNodes" ]; then
+                    writeErrNodes="${writeErrNodes}, "
+                fi
+                writeErrNodes="${writeErrNodes}${node}"
+            fi
+        else
+            echo "[WARN] Failed to detect WRITE errors count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect WRITE errors count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^WRITE speed" | sed -r "s/WRITE speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+        if [ -n "$cnt" ]; then
+            writeSpeed=$(( $writeSpeed+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] WRITE speed: $cnt msg/sec"
+            else
+                echo "[WARN] WRITE speed is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "WRITE speed is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect WRITE speed for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect WRITE speed for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^BULK_WRITE messages" | sed -r "s/BULK_WRITE messages: //g" | xargs)
+        if [ -n "$cnt" ]; then
+            blkWriteMsg=$(( $blkWriteMsg+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] BULK_WRITE messages: $cnt"
+            else
+                echo "[WARN] BULK_WRITE messages count is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "BULK_WRITE messages count is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect BULK_WRITE messages count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect BULK_WRITE messages count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^BULK_WRITE errors" | sed -r "s/BULK_WRITE errors: //g" | sed -r "s/,.*//g" | xargs)
+        if [ -n "$cnt" ]; then
+            blkWriteErrors=$(( $blkWriteErrors+$cnt ))
+            echo "[INFO] BULK_WRITE errors: $cnt"
+            if [ $cnt -ne 0 ]; then
+                if [ -n "$blkWriteErrNodes" ]; then
+                    blkWriteErrNodes="${blkWriteErrNodes}, "
+                fi
+                blkWriteErrNodes="${blkWriteErrNodes}${node}"
+            fi
+        else
+            echo "[WARN] Failed to detect BULK_WRITE errors count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect BULK_WRITE errors count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^BULK_WRITE speed" | sed -r "s/BULK_WRITE speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+        if [ -n "$cnt" ]; then
+            blkWriteSpeed=$(( $blkWriteSpeed+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] BULK_WRITE speed: $cnt msg/sec"
+            else
+                echo "[WARN] BULK_WRITE speed is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "BULK_WRITE speed is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect BULK_WRITE speed for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect BULK_WRITE speed for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^READ messages" | sed -r "s/READ messages: //g" | xargs)
+        if [ -n "$cnt" ]; then
+            readMsg=$(( $readMsg+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] READ messages: $cnt"
+            else
+                echo "[WARN] READ messages count is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "READ messages count is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect READ messages count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect READ messages count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^READ errors" | sed -r "s/READ errors: //g" | sed -r "s/,.*//g" | xargs)
+        if [ -n "$cnt" ]; then
+            readErrors=$(( $readErrors+$cnt ))
+            echo "[INFO] READ errors: $cnt"
+            if [ $cnt -ne 0 ]; then
+                if [ -n "$readErrNodes" ]; then
+                    blkWriteErrNodes="${readErrNodes}, "
+                fi
+                readErrNodes="${readErrNodes}${node}"
+            fi
+        else
+            echo "[WARN] Failed to detect READ errors count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect READ errors count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^READ speed" | sed -r "s/READ speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+        if [ -n "$cnt" ]; then
+            readSpeed=$(( $readSpeed+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] READ speed: $cnt msg/sec"
+            else
+                echo "[WARN] READ speed is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "READ speed is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect READ speed for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect READ speed for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^BULK_READ messages" | sed -r "s/BULK_READ messages: //g" | xargs)
+        if [ -n "$cnt" ]; then
+            blkReadMsg=$(( $blkReadMsg+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] BULK_READ messages: $cnt"
+            else
+                echo "[WARN] BULK_READ messages count is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "BULK_READ messages count is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect BULK_READ messages count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect BULK_READ messages count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^BULK_READ errors" | sed -r "s/BULK_READ errors: //g" | sed -r "s/,.*//g" | xargs)
+        if [ -n "$cnt" ]; then
+            blkReadErrors=$(( $blkReadErrors+$cnt ))
+            echo "[INFO] BULK_READ errors: $cnt"
+            if [ $cnt -ne 0 ]; then
+                if [ -n "$blkReadErrNodes" ]; then
+                    blkReadErrNodes="${blkReadErrNodes}, "
+                fi
+                blkReadErrNodes="${blkReadErrNodes}${node}"
+            fi
+        else
+            echo "[WARN] Failed to detect BULK_READ errors count for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect BULK_READ errors count for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+
+        cnt=$(cat $logFile | grep "^BULK_READ speed" | sed -r "s/BULK_READ speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+        if [ -n "$cnt" ]; then
+            blkReadSpeed=$(( $blkReadSpeed+$cnt ))
+            if [ $cnt -ne 0 ]; then
+                echo "[INFO] BULK_READ speed: $cnt msg/sec"
+            else
+                echo "[WARN] BULK_READ speed is zero for $node node. This test probably failed."
+                echo "WARNING |" >> $tmpFile
+                echo "--------" >> $tmpFile
+                echo "BULK_READ speed is zero for $node node. This test probably failed." >> $tmpFile
+                echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+            fi
+        else
+            echo "[WARN] Failed to detect BULK_READ speed for $node node. This test probably failed."
+            echo "WARNING |" >> $tmpFile
+            echo "--------" >> $tmpFile
+            echo "Failed to detect BULK_READ speed for $node node. This test probably failed." >> $tmpFile
+            echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+        fi
+    done
+
+    echo "-------------------------------------------------------------------------------------"
+
+    echo "WRITE test metrics |" >> $1
+    echo "-------------------" >> $1
+    echo "Messages: $writeMsg" >> $1
+    echo "Speed   : $writeSpeed msg/sec" >> $1
+    echo "Errors  : $writeErrors" >> $1
+    echo "----------------------------------------------------------------------------------------------" >> $1
+
+    echo "BULK_WRITE test metrics |" >> $1
+    echo "------------------------" >> $1
+    echo "Messages: $blkWriteMsg" >> $1
+    echo "Speed   : $blkWriteSpeed msg/sec" >> $1
+    echo "Errors  : $blkWriteErrors" >> $1
+    echo "----------------------------------------------------------------------------------------------" >> $1
+
+    echo "READ test metrics |" >> $1
+    echo "------------------" >> $1
+    echo "Messages: $readMsg" >> $1
+    echo "Speed   : $readSpeed msg/sec" >> $1
+    echo "Errors  : $readErrors" >> $1
+    echo "----------------------------------------------------------------------------------------------" >> $1
+
+    echo "BULK_READ test metrics |" >> $1
+    echo "-----------------------" >> $1
+    echo "Messages: $blkReadMsg" >> $1
+    echo "Speed   : $blkReadSpeed msg/sec" >> $1
+    echo "Errors  : $blkReadErrors" >> $1
+    echo "----------------------------------------------------------------------------------------------" >> $1
+
+    if [ -n "$writeErrNodes" ]; then
+        echo "Nodes having WRITE errors |" >> $1
+        echo "-------------------------------" >> $1
+        echo "$writeErrNodes" >> $1
+        echo "----------------------------------------------------------------------------------------------" >> $1
+    fi
+
+    if [ -n "$blkWriteErrNodes" ]; then
+        echo "Nodes having BULK_WRITE errors |" >> $1
+        echo "-------------------------------" >> $1
+        echo "$blkWriteErrNodes" >> $1
+        echo "----------------------------------------------------------------------------------------------" >> $1
+    fi
+
+    if [ -n "$readErrNodes" ]; then
+        echo "Nodes having READ errors |" >> $1
+        echo "-------------------------------" >> $1
+        echo "$readErrNodes" >> $1
+        echo "----------------------------------------------------------------------------------------------" >> $1
+    fi
+
+    if [ -n "$blkReadErrNodes" ]; then
+        echo "Nodes having BULK_READ errors |" >> $1
+        echo "-------------------------------" >> $1
+        echo "$blkReadErrNodes" >> $1
+        echo "----------------------------------------------------------------------------------------------" >> $1
+    fi
+
+    cat $tmpFile >> $1
+
+    rm -f $tmpFile
+}
+
+reportFailedTestsDetailes()
+{
+    for dir in $2/*
+    do
+        node=$(echo $dir | sed -r "s/^.*\///g")
+        if [ -z "$node" ]; then
+            continue
+        fi
+
+        echo "----------------------------------------------------------------------------------------------" >> $1
+        echo "Error details for node: $node" >> $1
+        echo "----------------------------------------------------------------------------------------------" >> $1
+
+        if [ -f "$dir/__error__" ]; then
+            cat $dir/__error__ >> $1
+        else
+            echo "N/A" >> $1
+        fi
+    done
+}
+
+validate
+reportTestsSummary
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/tests/tests-run.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/tests/tests-run.sh b/modules/cassandra/src/test/bootstrap/aws/tests/tests-run.sh
new file mode 100644
index 0000000..74a769a
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/tests/tests-run.sh
@@ -0,0 +1,715 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+
+terminate()
+{
+    if [[ "$S3_TESTS_SUCCESS_URL" != */ ]]; then
+        S3_TESTS_SUCCESS_URL=${S3_TESTS_SUCCESS_URL}/
+    fi
+
+    if [[ "$S3_TESTS_FAILURE_URL" != */ ]]; then
+        S3_TESTS_FAILURE_URL=${S3_TESTS_FAILURE_URL}/
+    fi
+
+    msg=$HOST_NAME
+
+    if [ -n "$1" ]; then
+        echo "[ERROR] $1"
+        echo "[ERROR]-----------------------------------------------------"
+        echo "[ERROR] Tests execution failed"
+        echo "[ERROR]-----------------------------------------------------"
+        msg=$1
+        reportFolder=${S3_TESTS_FAILURE_URL}${HOST_NAME}
+        reportFile=$reportFolder/__error__
+    else
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Tests execution successfully completed"
+        echo "[INFO]-----------------------------------------------------"
+        reportFolder=${S3_TESTS_SUCCESS_URL}${HOST_NAME}
+        reportFile=$reportFolder/__success__
+    fi
+
+    echo $msg > /opt/ignite-cassandra-tests/tests-result
+
+    aws s3 rm --recursive $reportFolder
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed drop report folder: $reportFolder"
+    fi
+
+    if [ -d "/opt/ignite-cassandra-tests/logs" ]; then
+        aws s3 sync --sse AES256 /opt/ignite-cassandra-tests/logs $reportFolder
+        if [ $? -ne 0 ]; then
+            echo "[ERROR] Failed to export tests logs to: $reportFolder"
+        fi
+    fi
+
+    aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/tests-result $reportFile
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to report tests results to: $reportFile"
+    fi
+
+    aws s3 rm ${S3_TESTS_RUNNING_URL}${HOST_NAME}
+    aws s3 rm ${S3_TESTS_WAITING_URL}${HOST_NAME}
+
+    if [ "$FIRST_NODE" == "true" ]; then
+        waitAllTestNodesCompleted
+        removeFirstNodeLock
+        reportScript=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/tests-report.sh)
+        $reportScript
+
+        if [ -n "$S3_LOGS_TRIGGER_URL" ]; then
+            aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/hostname $S3_LOGS_TRIGGER_URL
+            if [ $? -ne 0 ]; then
+                echo "[ERROR] Failed to trigger logs collection"
+            fi
+        fi
+    fi
+
+    rm -Rf /opt/ignite-cassandra-tests/tests-result /opt/ignite-cassandra-tests/hostname
+
+    if [ -n "$1" ]; then
+        exit 1
+    fi
+
+    exit 0
+}
+
+cleanupMetadata()
+{
+    echo "[INFO] Running cleanup"
+    aws s3 rm $S3_TESTS_SUMMARY_URL
+    aws s3 rm --recursive $S3_TEST_NODES_DISCOVERY_URL
+    aws s3 rm --recursive $S3_TESTS_RUNNING_URL
+    aws s3 rm --recursive $S3_TESTS_WAITING_URL
+    aws s3 rm --recursive $S3_TESTS_SUCCESS_URL
+    aws s3 rm --recursive $S3_TESTS_FAILURE_URL
+    echo "[INFO] Cleanup completed"
+}
+
+registerTestNode()
+{
+    aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/hostname ${S3_TEST_NODES_DISCOVERY_URL}${HOST_NAME}
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create test node registration flag: ${S3_TEST_NODES_DISCOVERY_URL}${HOST_NAME}"
+    fi
+}
+
+createRunningFlag()
+{
+    aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/hostname ${S3_TESTS_RUNNING_URL}${HOST_NAME}
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create tests running flag: ${S3_TESTS_RUNNING_URL}${HOST_NAME}"
+    fi
+}
+
+dropRunningFlag()
+{
+    exists=$(aws s3 ls ${S3_TESTS_RUNNING_URL}${HOST_NAME})
+    if [ -z "$exists" ]; then
+        return 0
+    fi
+
+    aws s3 rm ${S3_TESTS_RUNNING_URL}${HOST_NAME}
+    if [ $? -ne 0 ]; then
+        terminate "Failed to drop tests running flag: ${S3_TESTS_RUNNING_URL}${HOST_NAME}"
+    fi
+}
+
+createWaitingFlag()
+{
+    aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/hostname ${S3_TESTS_WAITING_URL}${HOST_NAME}
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create tests waiting flag: ${S3_TESTS_WAITING_URL}${HOST_NAME}"
+    fi
+}
+
+dropWaitingFlag()
+{
+    exists=$(aws s3 ls ${S3_TESTS_WAITING_URL}${HOST_NAME})
+    if [ -z "$exists" ]; then
+        return 0
+    fi
+
+    aws s3 rm ${S3_TESTS_WAITING_URL}${HOST_NAME}
+    if [ $? -ne 0 ]; then
+        terminate "Failed to drop tests waiting flag: ${S3_TESTS_WAITING_URL}${HOST_NAME}"
+    fi
+}
+
+dropTestsSummary()
+{
+    exists=$(aws s3 ls $S3_TESTS_SUMMARY_URL)
+    if [ -z "$exists" ]; then
+        return 0
+    fi
+
+    aws s3 rm $S3_TESTS_SUMMARY_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to drop tests summary info: $S3_TESTS_SUMMARY_URL"
+    fi
+}
+
+validate()
+{
+    if [ -z "$TESTS_TYPE" ]; then
+        terminate "Tests type 'ignite' or 'cassandra' should be specified"
+    fi
+
+    if [ "$TESTS_TYPE" != "ignite" ] && [ "$TESTS_TYPE" != "cassandra" ]; then
+        terminate "Incorrect tests type specified: $TESTS_TYPE"
+    fi
+
+    if [ -z "$S3_TESTS_SUCCESS_URL" ]; then
+        terminate "Tests success URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_SUCCESS_URL" != */ ]]; then
+        S3_TESTS_SUCCESS_URL=${S3_TESTS_SUCCESS_URL}/
+    fi
+
+    if [ -z "$S3_TESTS_FAILURE_URL" ]; then
+        terminate "Tests failure URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_FAILURE_URL" != */ ]]; then
+        S3_TESTS_FAILURE_URL=${S3_TESTS_FAILURE_URL}/
+    fi
+
+    if [ -z "$S3_TESTS_RUNNING_URL" ]; then
+        terminate "Tests running URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_RUNNING_URL" != */ ]]; then
+        S3_TESTS_RUNNING_URL=${S3_TESTS_RUNNING_URL}/
+    fi
+
+    if [ -z "$S3_TESTS_WAITING_URL" ]; then
+        terminate "Tests waiting URL doesn't specified"
+    fi
+
+    if [[ "$S3_TESTS_WAITING_URL" != */ ]]; then
+        S3_TESTS_WAITING_URL=${S3_TESTS_WAITING_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_SUCCESS_URL" ]; then
+        terminate "Ignite success URL doesn't specified"
+    fi
+
+    if [[ "$S3_IGNITE_SUCCESS_URL" != */ ]]; then
+        S3_IGNITE_SUCCESS_URL=${S3_IGNITE_SUCCESS_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_FAILURE_URL" ]; then
+        terminate "Ignite failure URL doesn't specified"
+    fi
+
+    if [[ "$S3_IGNITE_FAILURE_URL" != */ ]]; then
+        S3_IGNITE_FAILURE_URL=${S3_IGNITE_FAILURE_URL}/
+    fi
+
+    if [ -z "$S3_CASSANDRA_SUCCESS_URL" ]; then
+        terminate "Cassandra success URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_SUCCESS_URL" != */ ]]; then
+        S3_CASSANDRA_SUCCESS_URL=${S3_CASSANDRA_SUCCESS_URL}/
+    fi
+
+    if [ -z "$S3_CASSANDRA_FAILURE_URL" ]; then
+        terminate "Cassandra failure URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_FAILURE_URL" != */ ]]; then
+        S3_CASSANDRA_FAILURE_URL=${S3_CASSANDRA_FAILURE_URL}/
+    fi
+
+    if [ -z "$S3_TEST_NODES_DISCOVERY_URL" ]; then
+        terminate "Tests S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_TEST_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_TEST_NODES_DISCOVERY_URL=${S3_TEST_NODES_DISCOVERY_URL}/
+    fi
+
+    if [ -z "$S3_CASSANDRA_NODES_DISCOVERY_URL" ]; then
+        terminate "Cassandra S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_NODES_DISCOVERY_URL" ]; then
+        terminate "Ignite S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+    fi
+
+    if [ -z "$S3_IGNITE_NODES_DISCOVERY_URL" ]; then
+        terminate "Ignite S3 discovery URL doesn't specified"
+    fi
+
+    if [[ "$S3_IGNITE_NODES_DISCOVERY_URL" != */ ]]; then
+        S3_IGNITE_NODES_DISCOVERY_URL=${S3_IGNITE_NODES_DISCOVERY_URL}/
+    fi
+}
+
+setupCassandraSeeds()
+{
+    if [ $CASSANDRA_NODES_COUNT -eq 0 ]; then
+        return 0
+    fi
+
+    CASSANDRA_SEEDS1=
+    CASSANDRA_SEEDS2=
+
+    echo "[INFO] Setting up Cassandra seeds"
+
+    echo "[INFO] Looking for Cassandra seeds in: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+
+    startTime=$(date +%s)
+
+    while true; do
+        seeds=$(aws s3 ls $S3_CASSANDRA_NODES_DISCOVERY_URL | grep -v PRE | sed -r "s/^.* //g")
+        if [ -n "$seeds" ]; then
+            seeds=($seeds)
+            length=${#seeds[@]}
+
+            if [ $length -lt 4 ]; then
+                seed1=${seeds[0]}
+                seed2=${seeds[1]}
+                seed3=${seeds[2]}
+            else
+                pos1=$(($RANDOM%$length))
+                pos2=$(($RANDOM%$length))
+                pos3=$(($RANDOM%$length))
+                seed1=${seeds[${pos1}]}
+                seed2=${seeds[${pos2}]}
+                seed3=${seeds[${pos3}]}
+            fi
+
+            CASSANDRA_SEEDS1="<value>$seed1<\/value>"
+            CASSANDRA_SEEDS2="$seed1"
+
+            if [ "$seed2" != "$seed1" ] && [ -n "$seed2" ]; then
+                CASSANDRA_SEEDS1="$CASSANDRA_SEEDS1<value>$seed2<\/value>"
+                CASSANDRA_SEEDS2="${CASSANDRA_SEEDS2},$seed2"
+            fi
+
+            if [ "$seed3" != "$seed2" ] && [ "$seed3" != "$seed1" ] && [ -n "$seed3" ]; then
+                CASSANDRA_SEEDS1="$CASSANDRA_SEEDS1<value>$seed3<\/value>"
+                CASSANDRA_SEEDS2="${CASSANDRA_SEEDS2},$seed3"
+            fi
+
+            echo "[INFO] Using Cassandra seeds: $CASSANDRA_SEEDS2"
+
+            echo "contact.points=$CASSANDRA_SEEDS2" > /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/connection.properties
+
+            cat /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template.xml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CASSANDRA_SEEDS1/g" > /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template1.xml
+
+            return 0
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but no Cassandra nodes is up and running"
+        fi
+
+        echo "[INFO] Waiting for the first Cassandra node to start and publish its seed, time passed ${duration}min"
+
+        sleep 1m
+    done
+}
+
+setupIgniteSeeds()
+{
+    if [ $IGNITE_NODES_COUNT -eq 0 ]; then
+        return 0
+    fi
+
+    echo "[INFO] Setting up Ignite seeds"
+
+    echo "[INFO] Looking for Ignite seeds in: $S3_IGNITE_NODES_DISCOVERY_URL"
+
+    startTime=$(date +%s)
+
+    while true; do
+        seeds=$(aws s3 ls $S3_IGNITE_NODES_DISCOVERY_URL | grep -v PRE | sed -r "s/^.* //g")
+        if [ -n "$seeds" ]; then
+            seeds=($seeds)
+            length=${#seeds[@]}
+
+            if [ $length -lt 4 ]; then
+                seed1=${seeds[0]}
+                seed2=${seeds[1]}
+                seed3=${seeds[2]}
+            else
+                pos1=$(($RANDOM%$length))
+                pos2=$(($RANDOM%$length))
+                pos3=$(($RANDOM%$length))
+                seed1=${seeds[${pos1}]}
+                seed2=${seeds[${pos2}]}
+                seed3=${seeds[${pos3}]}
+            fi
+
+            IGNITE_SEEDS="<value>$seed1<\/value>"
+
+            if [ "$seed2" != "$seed1" ] && [ -n "$seed2" ]; then
+                IGNITE_SEEDS="$IGNITE_SEEDS<value>$seed2<\/value>"
+            fi
+
+            if [ "$seed3" != "$seed2" ] && [ "$seed3" != "$seed1" ] && [ -n "$seed3" ]; then
+                IGNITE_SEEDS="$IGNITE_SEEDS<value>$seed3<\/value>"
+            fi
+
+            echo "[INFO] Using Ignite seeds: $IGNITE_SEEDS"
+
+            cat /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template1.xml | sed -r "s/\\\$\{IGNITE_SEEDS\}/$IGNITE_SEEDS/g" > /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml
+            rm -f /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template1.xml
+
+            return 0
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but no Ignite nodes is up and running"
+        fi
+
+        echo "[INFO] Waiting for the first Ignite node to start and publish its seed, time passed ${duration}min"
+
+        sleep 1m
+    done
+}
+
+tryToGetFirstNodeLock()
+{
+    echo "[INFO] Trying to get first node lock"
+
+    checkFirstNodeLockExist
+    if [ $? -ne 0 ]; then
+        return 1
+    fi
+
+    createFirstNodeLock
+
+    sleep 5s
+
+    rm -Rf /opt/ignite-cassandra-tests/first-node-lock
+
+    aws s3 cp $S3_TESTS_FIRST_NODE_LOCK_URL /opt/ignite-cassandra-tests/first-node-lock
+    if [ $? -ne 0 ]; then
+        echo "[WARN] Failed to check just created first node lock"
+        return 1
+    fi
+
+    first_host=$(cat /opt/ignite-cassandra-tests/first-node-lock)
+
+    rm -f /opt/ignite-cassandra-tests/first-node-lock
+
+    if [ "$first_host" != "$HOST_NAME" ]; then
+        echo "[INFO] Node $first_host has discarded previously created first node lock"
+        return 1
+    fi
+
+    echo "[INFO] Congratulations, got first node lock"
+
+    return 0
+}
+
+checkFirstNodeLockExist()
+{
+    echo "[INFO] Checking for the first node lock"
+
+    lockExists=$(aws s3 ls $S3_TESTS_FIRST_NODE_LOCK_URL)
+    if [ -n "$lockExists" ]; then
+        echo "[INFO] First node lock already exists"
+        return 1
+    fi
+
+    echo "[INFO] First node lock doesn't exist yet"
+
+    return 0
+}
+
+createFirstNodeLock()
+{
+    aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/hostname $S3_TESTS_FIRST_NODE_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create first node lock"
+    fi
+    echo "[INFO] Created first node lock"
+}
+
+removeFirstNodeLock()
+{
+    exists=$(aws s3 ls $S3_TESTS_FIRST_NODE_LOCK_URL)
+    if [ -z "$exists" ]; then
+        return 0
+    fi
+
+    aws s3 rm $S3_TESTS_FIRST_NODE_LOCK_URL
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to remove first node lock"
+        return 1
+    fi
+
+    echo "[INFO] Removed first node lock"
+}
+
+waitAllIgniteNodesReady()
+{
+    if [ $IGNITE_NODES_COUNT -eq 0 ]; then
+        return 0
+    fi
+
+    echo "[INFO] Waiting for all $IGNITE_NODES_COUNT Ignite nodes up and running"
+
+    while true; do
+        successCount=$(aws s3 ls $S3_IGNITE_SUCCESS_URL | wc -l)
+        failureCount=$(aws s3 ls $S3_IGNITE_FAILURE_URL | wc -l)
+
+        if [ $successCount -ge $IGNITE_NODES_COUNT ]; then
+            break
+        fi
+
+        if [ "$failureCount" != "0" ]; then
+            terminate "$failureCount Ignite nodes are failed to start. Thus it doesn't make sense to run tests."
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] Congratulation, all $IGNITE_NODES_COUNT Ignite nodes are up and running"
+}
+
+waitAllCassandraNodesReady()
+{
+    if [ $CASSANDRA_NODES_COUNT -eq 0 ]; then
+        return 0
+    fi
+
+    echo "[INFO] Waiting for all $CASSANDRA_NODES_COUNT Cassandra nodes up and running"
+
+    while true; do
+        successCount=$(aws s3 ls $S3_CASSANDRA_SUCCESS_URL | wc -l)
+        failureCount=$(aws s3 ls $S3_CASSANDRA_FAILURE_URL | wc -l)
+
+        if [ $successCount -ge $CASSANDRA_NODES_COUNT ]; then
+            break
+        fi
+
+        if [ "$failureCount" != "0" ]; then
+            terminate "$failureCount Cassandra nodes are failed to start. Thus it doesn't make sense to run tests."
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] Congratulation, all $CASSANDRA_NODES_COUNT Cassandra nodes are up and running"
+}
+
+waitFirstTestNodeRegistered()
+{
+    echo "[INFO] Waiting for the first test node to register"
+
+    while true; do
+        first_host=
+
+        exists=$(aws s3 ls $S3_TESTS_FIRST_NODE_LOCK_URL)
+        if [ -n "$exists" ]; then
+            rm -Rf /opt/ignite-cassandra-tests/first-node-lock
+
+            aws s3 cp $S3_TESTS_FIRST_NODE_LOCK_URL /opt/ignite-cassandra-tests/first-node-lock
+            if [ $? -ne 0 ]; then
+                terminate "Failed to check existing first node lock"
+            fi
+
+            first_host=$(cat /opt/ignite-cassandra-tests/first-node-lock)
+
+            rm -Rf /opt/ignite-cassandra-tests/first-node-lock
+        fi
+
+        if [ -n "$first_host" ]; then
+            exists=$(aws s3 ls ${S3_TEST_NODES_DISCOVERY_URL}${first_host})
+            if [ -n "$exists" ]; then
+                break
+            fi
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] First test node registered"
+}
+
+waitAllTestNodesReady()
+{
+    createWaitingFlag
+
+    echo "[INFO] Waiting for all $TEST_NODES_COUNT test nodes up and running"
+
+    while true; do
+
+        nodesCount=$(aws s3 ls $S3_TEST_NODES_DISCOVERY_URL | wc -l)
+
+        if [ $nodesCount -ge $TEST_NODES_COUNT ]; then
+            break
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] Congratulation, all $TEST_NODES_COUNT test nodes are up and running"
+
+    dropWaitingFlag
+    createRunningFlag
+}
+
+waitAllTestNodesCompleted()
+{
+    echo "[INFO] Waiting for all $TEST_NODES_COUNT test nodes to complete their tests"
+
+    while true; do
+        successCount=$(aws s3 ls $S3_TESTS_SUCCESS_URL | wc -l)
+        failureCount=$(aws s3 ls $S3_TESTS_FAILURE_URL | wc -l)
+        count=$(( $successCount+$failureCount ))
+
+        if [ $count -ge $TEST_NODES_COUNT ]; then
+            break
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] Congratulation, all $TEST_NODES_COUNT test nodes have completed their tests"
+}
+
+# Time (in minutes) to wait for Ignite/Cassandra node up and running and register it in S3
+NODE_STARTUP_TIME=10
+
+HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+echo $HOST_NAME > /opt/ignite-cassandra-tests/hostname
+
+validate
+
+FIRST_NODE="false"
+
+tryToGetFirstNodeLock
+
+if [ $? -eq 0 ]; then
+    FIRST_NODE="true"
+fi
+
+dropRunningFlag
+dropWaitingFlag
+
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE" == "true" ]; then
+    echo "[INFO] Running tests from first node"
+    dropTestsSummary
+else
+    echo "[INFO] Running tests"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Tests type: $TESTS_TYPE"
+echo "[INFO] Test nodes count: $TEST_NODES_COUNT"
+echo "[INFO] Ignite nodes count: $IGNITE_NODES_COUNT"
+echo "[INFO] Cassandra nodes count: $CASSANDRA_NODES_COUNT"
+echo "[INFO] Tests summary URL: $S3_TESTS_SUMMARY_URL"
+echo "[INFO] Tests first node lock URL: $S3_TESTS_FIRST_NODE_LOCK_URL"
+echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+echo "[INFO] Test node discovery URL: $S3_TEST_NODES_DISCOVERY_URL"
+echo "[INFO] Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY_URL"
+echo "[INFO] Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+echo "[INFO] Tests running URL: $S3_TESTS_RUNNING_URL"
+echo "[INFO] Tests waiting URL: $S3_TESTS_WAITING_URL"
+echo "[INFO] Tests success URL: $S3_TESTS_SUCCESS_URL"
+echo "[INFO] Tests failure URL: $S3_TESTS_FAILURE_URL"
+echo "[INFO] Ignite success URL: $S3_IGNITE_SUCCESS_URL"
+echo "[INFO] Ignite failure URL: $S3_IGNITE_FAILURE_URL"
+echo "[INFO] Cassandra success URL: $S3_CASSANDRA_SUCCESS_URL"
+echo "[INFO] Cassandra failure URL: $S3_CASSANDRA_FAILURE_URL"
+echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER_URL"
+echo "[INFO] JAVA_HOME: $JAVA_HOME"
+echo "[INFO] PATH: $PATH"
+echo "[INFO]-----------------------------------------------------------------"
+
+echo "admin.user=cassandra" > /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+echo "admin.password=cassandra" >> /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+echo "regular.user=cassandra" >> /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+echo "regular.password=cassandra" >> /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+
+waitAllCassandraNodesReady
+waitAllIgniteNodesReady
+
+setupCassandraSeeds
+setupIgniteSeeds
+
+if [ "$FIRST_NODE" != "true" ]; then
+    waitFirstTestNodeRegistered
+else
+    cleanupMetadata
+fi
+
+registerTestNode
+
+waitAllTestNodesReady
+
+cd /opt/ignite-cassandra-tests
+
+if [ "$TESTS_TYPE" == "ignite" ]; then
+    echo "[INFO] Running Ignite load tests"
+    ./ignite-load-tests.sh
+    result=$?
+else
+    echo "[INFO] Running Cassandra load tests"
+    ./cassandra-load-tests.sh
+    result=$?
+fi
+
+if [ $result -ne 0 ]; then
+    terminate ""
+fi
+
+terminate

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java
new file mode 100644
index 0000000..c031bfb
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.tests;
+
+import org.apache.ignite.tests.load.LoadTestDriver;
+import org.apache.ignite.tests.load.cassandra.BulkReadWorker;
+import org.apache.ignite.tests.load.cassandra.BulkWriteWorker;
+import org.apache.ignite.tests.load.cassandra.ReadWorker;
+import org.apache.ignite.tests.load.cassandra.WriteWorker;
+import org.apache.ignite.tests.utils.CacheStoreHelper;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.log4j.Logger;
+
+/**
+ * Load tests for {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore} implementation of
+ * {@link org.apache.ignite.cache.store.CacheStore} which allows to store Ignite cache data into Cassandra tables.
+ */
+public class CassandraDirectPersistenceLoadTest extends LoadTestDriver {
+    /** */
+    private static final Logger LOGGER = Logger.getLogger("CassandraLoadTests");
+
+    /**
+     *
+     * @param args Test arguments.
+     */
+    public static void main(String[] args) {
+        try {
+            LOGGER.info("Cassandra load tests execution started");
+
+            LoadTestDriver driver = new CassandraDirectPersistenceLoadTest();
+
+            /**
+             * Load test scripts could be executed from several machines. Current implementation can correctly,
+             * handle situation when Cassandra keyspace/table was dropped - for example by the same load test
+             * started a bit later on another machine. Moreover there is a warm up period for each load test.
+             * Thus all the delays related to keyspaces/tables recreation actions will not affect performance metrics,
+             * but it will be produced lots of "trash" output in the logs (related to correct handling of such
+             * exceptional situation and keyspace/table recreation).
+             *
+             * Thus dropping test keyspaces at the beginning of the tests makes sense only for Unit tests,
+             * but not for Load tests.
+            **/
+
+            //CassandraHelper.dropTestKeyspaces();
+
+            driver.runTest("WRITE", WriteWorker.class, WriteWorker.LOGGER_NAME);
+
+            driver.runTest("BULK_WRITE", BulkWriteWorker.class, BulkWriteWorker.LOGGER_NAME);
+
+            driver.runTest("READ", ReadWorker.class, ReadWorker.LOGGER_NAME);
+
+            driver.runTest("BULK_READ", BulkReadWorker.class, BulkReadWorker.LOGGER_NAME);
+
+            /**
+             * Load test script executed on one machine could complete earlier that the same load test executed from
+             * another machine. Current implementation can correctly handle situation when Cassandra keyspace/table
+             * was dropped (simply recreate it). But dropping keyspace/table during load tests execution and subsequent
+             * recreation of such objects can have SIGNIFICANT EFFECT on final performance metrics.
+             *
+             * Thus dropping test keyspaces at the end of the tests makes sense only for Unit tests,
+             * but not for Load tests.
+             */
+
+            //CassandraHelper.dropTestKeyspaces(); // REVIEW This line is commented by purpose?
+
+            LOGGER.info("Cassandra load tests execution completed");
+        }
+        catch (Throwable e) {
+            LOGGER.error("Cassandra load tests execution failed", e);
+            throw new RuntimeException("Cassandra load tests execution failed", e);
+        }
+        finally {
+            CassandraHelper.releaseCassandraResources();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Logger logger() {
+        return LOGGER;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object setup(String logName) {
+        return CacheStoreHelper.createCacheStore(
+            TestsHelper.getLoadTestsCacheName(),
+            TestsHelper.getLoadTestsPersistenceSettings(),
+            CassandraHelper.getAdminDataSrc(),
+            Logger.getLogger(logName));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java
new file mode 100644
index 0000000..26cca68
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java
@@ -0,0 +1,371 @@
+/*
+ * 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.tests;
+
+import java.util.Collection;
+import java.util.Map;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.pojos.Person;
+import org.apache.ignite.tests.pojos.PersonId;
+import org.apache.ignite.tests.utils.CacheStoreHelper;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.springframework.core.io.ClassPathResource;
+
+/**
+ * Unit tests for {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore} implementation of
+ * {@link org.apache.ignite.cache.store.CacheStore} which allows to store Ignite cache data into Cassandra tables.
+ */
+public class CassandraDirectPersistenceTest {
+    /** */
+    private static final Logger LOGGER = Logger.getLogger(CassandraDirectPersistenceTest.class.getName());
+
+    /** */
+    @BeforeClass
+    public static void setUpClass() {
+        if (CassandraHelper.useEmbeddedCassandra()) {
+            try {
+                CassandraHelper.startEmbeddedCassandra(LOGGER);
+            }
+            catch (Throwable e) {
+                throw new RuntimeException("Failed to start embedded Cassandra instance", e);
+            }
+        }
+
+        LOGGER.info("Testing admin connection to Cassandra");
+        CassandraHelper.testAdminConnection();
+
+        LOGGER.info("Testing regular connection to Cassandra");
+        CassandraHelper.testRegularConnection();
+
+        LOGGER.info("Dropping all artifacts from previous tests execution session");
+        CassandraHelper.dropTestKeyspaces();
+
+        LOGGER.info("Start tests execution");
+    }
+
+    /** */
+    @AfterClass
+    public static void tearDownClass() {
+        try {
+            CassandraHelper.dropTestKeyspaces();
+        }
+        finally {
+            CassandraHelper.releaseCassandraResources();
+
+            if (CassandraHelper.useEmbeddedCassandra()) {
+                try {
+                    CassandraHelper.stopEmbeddedCassandra();
+                }
+                catch (Throwable e) {
+                    LOGGER.error("Failed to stop embedded Cassandra instance", e);
+                }
+            }
+        }
+    }
+
+    /** */
+    @Test
+    @SuppressWarnings("unchecked")
+    public void primitiveStrategyTest() {
+        CacheStore store1 = CacheStoreHelper.createCacheStore("longTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        CacheStore store2 = CacheStoreHelper.createCacheStore("stringTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        Collection<CacheEntryImpl<Long, Long>> longEntries = TestsHelper.generateLongsEntries();
+        Collection<CacheEntryImpl<String, String>> strEntries = TestsHelper.generateStringsEntries();
+
+        Collection<Long> fakeLongKeys = TestsHelper.getKeys(longEntries);
+        fakeLongKeys.add(-1L);
+        fakeLongKeys.add(-2L);
+        fakeLongKeys.add(-3L);
+        fakeLongKeys.add(-4L);
+
+        Collection<String> fakeStrKeys = TestsHelper.getKeys(strEntries);
+        fakeStrKeys.add("-1");
+        fakeStrKeys.add("-2");
+        fakeStrKeys.add("-3");
+        fakeStrKeys.add("-4");
+
+        LOGGER.info("Running PRIMITIVE strategy write tests");
+
+        LOGGER.info("Running single operation write tests");
+        store1.write(longEntries.iterator().next());
+        store2.write(strEntries.iterator().next());
+        LOGGER.info("Single operation write tests passed");
+
+        LOGGER.info("Running bulk operation write tests");
+        store1.writeAll(longEntries);
+        store2.writeAll(strEntries);
+        LOGGER.info("Bulk operation write tests passed");
+
+        LOGGER.info("PRIMITIVE strategy write tests passed");
+
+        LOGGER.info("Running PRIMITIVE strategy read tests");
+
+        LOGGER.info("Running single operation read tests");
+
+        LOGGER.info("Running real keys read tests");
+
+        Long longVal = (Long)store1.load(longEntries.iterator().next().getKey());
+        if (!longEntries.iterator().next().getValue().equals(longVal))
+            throw new RuntimeException("Long values was incorrectly deserialized from Cassandra");
+
+        String strVal = (String)store2.load(strEntries.iterator().next().getKey());
+        if (!strEntries.iterator().next().getValue().equals(strVal))
+            throw new RuntimeException("String values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Running fake keys read tests");
+
+        longVal = (Long)store1.load(-1L);
+        if (longVal != null)
+            throw new RuntimeException("Long value with fake key '-1' was found in Cassandra");
+
+        strVal = (String)store2.load("-1");
+        if (strVal != null)
+            throw new RuntimeException("String value with fake key '-1' was found in Cassandra");
+
+        LOGGER.info("Single operation read tests passed");
+
+        LOGGER.info("Running bulk operation read tests");
+
+        LOGGER.info("Running real keys read tests");
+
+        Map longValues = store1.loadAll(TestsHelper.getKeys(longEntries));
+        if (!TestsHelper.checkCollectionsEqual(longValues, longEntries))
+            throw new RuntimeException("Long values was incorrectly deserialized from Cassandra");
+
+        Map strValues = store2.loadAll(TestsHelper.getKeys(strEntries));
+        if (!TestsHelper.checkCollectionsEqual(strValues, strEntries))
+            throw new RuntimeException("String values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Running fake keys read tests");
+
+        longValues = store1.loadAll(fakeLongKeys);
+        if (!TestsHelper.checkCollectionsEqual(longValues, longEntries))
+            throw new RuntimeException("Long values was incorrectly deserialized from Cassandra");
+
+        strValues = store2.loadAll(fakeStrKeys);
+        if (!TestsHelper.checkCollectionsEqual(strValues, strEntries))
+            throw new RuntimeException("String values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Bulk operation read tests passed");
+
+        LOGGER.info("PRIMITIVE strategy read tests passed");
+
+        LOGGER.info("Running PRIMITIVE strategy delete tests");
+
+        LOGGER.info("Deleting real keys");
+
+        store1.delete(longEntries.iterator().next().getKey());
+        store1.deleteAll(TestsHelper.getKeys(longEntries));
+
+        store2.delete(strEntries.iterator().next().getKey());
+        store2.deleteAll(TestsHelper.getKeys(strEntries));
+
+        LOGGER.info("Deleting fake keys");
+
+        store1.delete(-1L);
+        store2.delete("-1");
+
+        store1.deleteAll(fakeLongKeys);
+        store2.deleteAll(fakeStrKeys);
+
+        LOGGER.info("PRIMITIVE strategy delete tests passed");
+    }
+
+    /** */
+    @Test
+    @SuppressWarnings("unchecked")
+    public void blobStrategyTest() {
+        CacheStore store1 = CacheStoreHelper.createCacheStore("longTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        CacheStore store2 = CacheStoreHelper.createCacheStore("personTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        CacheStore store3 = CacheStoreHelper.createCacheStore("personTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        Collection<CacheEntryImpl<Long, Long>> longEntries = TestsHelper.generateLongsEntries();
+        Collection<CacheEntryImpl<Long, Person>> personEntries = TestsHelper.generateLongsPersonsEntries();
+
+        LOGGER.info("Running BLOB strategy write tests");
+
+        LOGGER.info("Running single operation write tests");
+        store1.write(longEntries.iterator().next());
+        store2.write(personEntries.iterator().next());
+        store3.write(personEntries.iterator().next());
+        LOGGER.info("Single operation write tests passed");
+
+        LOGGER.info("Running bulk operation write tests");
+        store1.writeAll(longEntries);
+        store2.writeAll(personEntries);
+        store3.writeAll(personEntries);
+        LOGGER.info("Bulk operation write tests passed");
+
+        LOGGER.info("BLOB strategy write tests passed");
+
+        LOGGER.info("Running BLOB strategy read tests");
+
+        LOGGER.info("Running single operation read tests");
+
+        Long longVal = (Long)store1.load(longEntries.iterator().next().getKey());
+        if (!longEntries.iterator().next().getValue().equals(longVal))
+            throw new RuntimeException("Long values was incorrectly deserialized from Cassandra");
+
+        Person personVal = (Person)store2.load(personEntries.iterator().next().getKey());
+        if (!personEntries.iterator().next().getValue().equals(personVal))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        personVal = (Person)store3.load(personEntries.iterator().next().getKey());
+        if (!personEntries.iterator().next().getValue().equals(personVal))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Single operation read tests passed");
+
+        LOGGER.info("Running bulk operation read tests");
+
+        Map longValues = store1.loadAll(TestsHelper.getKeys(longEntries));
+        if (!TestsHelper.checkCollectionsEqual(longValues, longEntries))
+            throw new RuntimeException("Long values was incorrectly deserialized from Cassandra");
+
+        Map personValues = store2.loadAll(TestsHelper.getKeys(personEntries));
+        if (!TestsHelper.checkPersonCollectionsEqual(personValues, personEntries, false))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        personValues = store3.loadAll(TestsHelper.getKeys(personEntries));
+        if (!TestsHelper.checkPersonCollectionsEqual(personValues, personEntries, false))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Bulk operation read tests passed");
+
+        LOGGER.info("BLOB strategy read tests passed");
+
+        LOGGER.info("Running BLOB strategy delete tests");
+
+        store1.delete(longEntries.iterator().next().getKey());
+        store1.deleteAll(TestsHelper.getKeys(longEntries));
+
+        store2.delete(personEntries.iterator().next().getKey());
+        store2.deleteAll(TestsHelper.getKeys(personEntries));
+
+        store3.delete(personEntries.iterator().next().getKey());
+        store3.deleteAll(TestsHelper.getKeys(personEntries));
+
+        LOGGER.info("BLOB strategy delete tests passed");
+    }
+
+    /** */
+    @Test
+    @SuppressWarnings("unchecked")
+    public void pojoStrategyTest() {
+        CacheStore store1 = CacheStoreHelper.createCacheStore("longTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        CacheStore store2 = CacheStoreHelper.createCacheStore("personTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        CacheStore store3 = CacheStoreHelper.createCacheStore("personTypes",
+            new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        Collection<CacheEntryImpl<Long, Person>> entries1 = TestsHelper.generateLongsPersonsEntries();
+        Collection<CacheEntryImpl<PersonId, Person>> entries2 = TestsHelper.generatePersonIdsPersonsEntries();
+        Collection<CacheEntryImpl<PersonId, Person>> entries3 = TestsHelper.generatePersonIdsPersonsEntries();
+
+        LOGGER.info("Running POJO strategy write tests");
+
+        LOGGER.info("Running single operation write tests");
+        store1.write(entries1.iterator().next());
+        store2.write(entries2.iterator().next());
+        store3.write(entries3.iterator().next());
+        LOGGER.info("Single operation write tests passed");
+
+        LOGGER.info("Running bulk operation write tests");
+        store1.writeAll(entries1);
+        store2.writeAll(entries2);
+        store3.writeAll(entries3);
+        LOGGER.info("Bulk operation write tests passed");
+
+        LOGGER.info("POJO strategy write tests passed");
+
+        LOGGER.info("Running POJO strategy read tests");
+
+        LOGGER.info("Running single operation read tests");
+
+        Person person = (Person)store1.load(entries1.iterator().next().getKey());
+        if (!entries1.iterator().next().getValue().equalsPrimitiveFields(person))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        person = (Person)store2.load(entries2.iterator().next().getKey());
+        if (!entries2.iterator().next().getValue().equalsPrimitiveFields(person))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        person = (Person)store3.load(entries3.iterator().next().getKey());
+        if (!entries3.iterator().next().getValue().equals(person))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Single operation read tests passed");
+
+        LOGGER.info("Running bulk operation read tests");
+
+        Map persons = store1.loadAll(TestsHelper.getKeys(entries1));
+        if (!TestsHelper.checkPersonCollectionsEqual(persons, entries1, true))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        persons = store2.loadAll(TestsHelper.getKeys(entries2));
+        if (!TestsHelper.checkPersonCollectionsEqual(persons, entries2, true))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        persons = store3.loadAll(TestsHelper.getKeys(entries3));
+        if (!TestsHelper.checkPersonCollectionsEqual(persons, entries3, false))
+            throw new RuntimeException("Person values was incorrectly deserialized from Cassandra");
+
+        LOGGER.info("Bulk operation read tests passed");
+
+        LOGGER.info("POJO strategy read tests passed");
+
+        LOGGER.info("Running POJO strategy delete tests");
+
+        store1.delete(entries1.iterator().next().getKey());
+        store1.deleteAll(TestsHelper.getKeys(entries1));
+
+        store2.delete(entries2.iterator().next().getKey());
+        store2.deleteAll(TestsHelper.getKeys(entries2));
+
+        store3.delete(entries3.iterator().next().getKey());
+        store3.deleteAll(TestsHelper.getKeys(entries3));
+
+        LOGGER.info("POJO strategy delete tests passed");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java
new file mode 100644
index 0000000..5de3097
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.tests;
+
+import java.net.URL;
+import org.apache.ignite.cache.store.cassandra.utils.DDLGenerator;
+import org.junit.Test;
+
+/**
+ * DDLGenerator test.
+ */
+public class DDLGeneratorTest {
+    @Test
+    @SuppressWarnings("unchecked")
+    /** */
+    public void generatorTest() {
+        ClassLoader clsLdr = DDLGeneratorTest.class.getClassLoader();
+
+        URL url1 = clsLdr.getResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml");
+        String file1 = url1.getFile(); // TODO IGNITE-1371 Possible NPE
+
+        URL url2 = clsLdr.getResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml");
+        String file2 = url2.getFile();  // TODO IGNITE-1371 Possible NPE
+
+        DDLGenerator.main(new String[]{file1, file2});
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java
new file mode 100644
index 0000000..bfcf751
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.tests;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.tests.load.LoadTestDriver;
+import org.apache.ignite.tests.load.ignite.BulkReadWorker;
+import org.apache.ignite.tests.load.ignite.BulkWriteWorker;
+import org.apache.ignite.tests.load.ignite.ReadWorker;
+import org.apache.ignite.tests.load.ignite.WriteWorker;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.log4j.Logger;
+
+/**
+ * Load tests for Ignite caches which utilizing {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}
+ * to store cache data into Cassandra tables
+ */
+public class IgnitePersistentStoreLoadTest extends LoadTestDriver {
+    /** */
+    private static final Logger LOGGER = Logger.getLogger("IgniteLoadTests");
+
+    /**
+     * test starter.
+     *
+     * @param args Test arguments.
+     */
+    public static void main(String[] args) {
+        try {
+            LOGGER.info("Ignite load tests execution started");
+
+            LoadTestDriver driver = new IgnitePersistentStoreLoadTest();
+
+            /**
+             * Load test scripts could be executed from several machines. Current implementation can correctly,
+             * handle situation when Cassandra keyspace/table was dropped - for example by the same load test
+             * started a bit later on another machine. Moreover there is a warm up period for each load test.
+             * Thus all the delays related to keyspaces/tables recreation actions will not affect performance metrics,
+             * but it will be produced lots of "trash" output in the logs (related to correct handling of such
+             * exceptional situation and keyspace/table recreation).
+             *
+             * Thus dropping test keyspaces makes sense only for Unit tests, but not for Load tests.
+            **/
+
+            //CassandraHelper.dropTestKeyspaces();
+
+            driver.runTest("WRITE", WriteWorker.class, WriteWorker.LOGGER_NAME);
+
+            driver.runTest("BULK_WRITE", BulkWriteWorker.class, BulkWriteWorker.LOGGER_NAME);
+
+            driver.runTest("READ", ReadWorker.class, ReadWorker.LOGGER_NAME);
+
+            driver.runTest("BULK_READ", BulkReadWorker.class, BulkReadWorker.LOGGER_NAME);
+
+            /**
+             * Load test script executed on one machine could complete earlier that the same load test executed from
+             * another machine. Current implementation can correctly handle situation when Cassandra keyspace/table
+             * was dropped (simply recreate it). But dropping keyspace/table during load tests execution and subsequent
+             * recreation of such objects can have SIGNIFICANT EFFECT on final performance metrics.
+             *
+             * Thus dropping test keyspaces at the end of the tests makes sense only for Unit tests,
+             * but not for Load tests.
+             */
+
+            //CassandraHelper.dropTestKeyspaces();
+
+            LOGGER.info("Ignite load tests execution completed");
+        }
+        catch (Throwable e) {
+            LOGGER.error("Ignite load tests execution failed", e);
+            throw new RuntimeException("Ignite load tests execution failed", e);
+        }
+        finally {
+            CassandraHelper.releaseCassandraResources();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Logger logger() {
+        return LOGGER;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Object setup(String logName) {
+        return Ignition.start(TestsHelper.getLoadTestsIgniteConfig());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void tearDown(Object obj) {
+        Ignite ignite = (Ignite)obj;
+
+        if (ignite != null)
+            ignite.close();
+    }
+}


[11/50] ignite git commit: ignite-3209 Waiting for affinity topology in case of failover for affinity call

Posted by vo...@apache.org.
ignite-3209 Waiting for affinity topology in case of failover for affinity call


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

Branch: refs/heads/ignite-3341
Commit: 1d80a398a40dd8b4b86bb83499cf0efd1f35b82b
Parents: 54425bf
Author: agura <ag...@gridgain.com>
Authored: Tue Jun 14 17:32:25 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jun 14 18:42:17 2016 +0300

----------------------------------------------------------------------
 .../processors/task/GridTaskWorker.java         | 130 ++++++++++++-------
 .../cache/CacheAffinityCallSelfTest.java        |   4 -
 2 files changed, 81 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1d80a398/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index dc86343..651259d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -61,12 +61,15 @@ import org.apache.ignite.internal.GridJobSiblingImpl;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTaskSessionImpl;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.compute.ComputeTaskTimeoutCheckedException;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.closure.AffinityTask;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
@@ -76,6 +79,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException;
+import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.resources.TaskContinuousMapperResource;
@@ -847,8 +852,25 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                         }
 
                         case FAILOVER: {
-                            if (!failover(res, jobRes, getTaskTopology()))
-                                plc = null;
+                            IgniteInternalFuture<Boolean> fut = failover(res, jobRes, getTaskTopology());
+
+                            final GridJobResultImpl jobRes0 = jobRes;
+
+                            fut.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
+                                @Override public void apply(IgniteInternalFuture<Boolean> fut0) {
+                                    try {
+                                        Boolean res = fut0.get();
+
+                                        if (res)
+                                            sendFailoverRequest(jobRes0);
+                                    }
+                                    catch (IgniteCheckedException e) {
+                                        U.error(log, "Failed to failover task [ses=" + ses + ", err=" + e + ']', e);
+
+                                        finishTask(null, e);
+                                    }
+                                }
+                            });
 
                             break;
                         }
@@ -856,16 +878,11 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                 }
 
                 // Outside of synchronization.
-                if (plc != null) {
-                    // Handle failover.
-                    if (plc == FAILOVER)
-                        sendFailoverRequest(jobRes);
-                    else {
-                        evtLsnr.onJobFinished(this, jobRes.getSibling());
+                if (plc != null && plc != FAILOVER) {
+                    evtLsnr.onJobFinished(this, jobRes.getSibling());
 
-                        if (plc == ComputeJobResultPolicy.REDUCE)
-                            reduce(results);
-                    }
+                    if (plc == ComputeJobResultPolicy.REDUCE)
+                        reduce(results);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -1039,59 +1056,74 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
      * @param top Topology.
      * @return {@code True} if fail-over SPI returned a new node.
      */
-    private boolean failover(GridJobExecuteResponse res, GridJobResultImpl jobRes, Collection<? extends ClusterNode> top) {
-        assert Thread.holdsLock(mux);
+    private IgniteInternalFuture<Boolean> failover(
+        final GridJobExecuteResponse res,
+        final GridJobResultImpl jobRes,
+        final Collection<? extends ClusterNode> top
+    ) {
+        IgniteInternalFuture<?> affFut = null;
 
-        try {
-            ctx.resource().invokeAnnotated(dep, jobRes.getJob(), ComputeJobBeforeFailover.class);
+        if (affKey != null) {
+            Long topVer = ctx.discovery().topologyVersion();
 
-            // Map to a new node.
-            ClusterNode node = ctx.failover().failover(ses, jobRes, new ArrayList<>(top), affKey, affCache);
+            affFut = ctx.cache().context().exchange().affinityReadyFuture(new AffinityTopologyVersion(topVer));
+        }
 
-            if (node == null) {
-                String msg = "Failed to failover a job to another node (failover SPI returned null) [job=" +
-                    jobRes.getJob() + ", node=" + jobRes.getNode() + ']';
+        if (affFut == null)
+            affFut = new GridFinishedFuture();
 
-                if (log.isDebugEnabled())
-                    log.debug(msg);
+        return affFut.chain(new IgniteClosure<IgniteInternalFuture<?>, Boolean>() {
+            @Override public Boolean apply(IgniteInternalFuture<?> fut0) {
+                synchronized (mux) {
+                    try {
+                        ctx.resource().invokeAnnotated(dep, jobRes.getJob(), ComputeJobBeforeFailover.class);
 
-                Throwable e = new ClusterTopologyCheckedException(msg, jobRes.getException());
+                        // Map to a new node.
+                        ClusterNode node = ctx.failover().failover(ses, jobRes, new ArrayList<>(top), affKey, affCache);
 
-                finishTask(null, e);
+                        if (node == null) {
+                            String msg = "Failed to failover a job to another node (failover SPI returned null) [job=" +
+                                jobRes.getJob() + ", node=" + jobRes.getNode() + ']';
 
-                return false;
-            }
+                            if (log.isDebugEnabled())
+                                log.debug(msg);
 
-            if (log.isDebugEnabled())
-                log.debug("Resolved job failover [newNode=" + node + ", oldNode=" + jobRes.getNode() +
-                    ", job=" + jobRes.getJob() + ", resMsg=" + res + ']');
+                            Throwable e = new ClusterTopologyCheckedException(msg, jobRes.getException());
 
-            jobRes.setNode(node);
-            jobRes.resetResponse();
+                            finishTask(null, e);
 
-            if (!resCache) {
-                synchronized (mux) {
-                    // Store result back in map before sending.
-                    this.jobRes.put(res.getJobId(), jobRes);
-                }
-            }
+                            return false;
+                        }
 
-            return true;
-        }
-        // Catch Throwable to protect against bad user code.
-        catch (Throwable e) {
-            String errMsg = "Failed to failover job due to undeclared user exception [job=" +
-                jobRes.getJob() + ", err=" + e + ']';
+                        if (log.isDebugEnabled())
+                            log.debug("Resolved job failover [newNode=" + node + ", oldNode=" + jobRes.getNode() +
+                                ", job=" + jobRes.getJob() + ", resMsg=" + res + ']');
 
-            U.error(log, errMsg, e);
+                        jobRes.setNode(node);
+                        jobRes.resetResponse();
 
-            finishTask(null, new ComputeUserUndeclaredException(errMsg, e));
+                        if (!resCache) // Store result back in map before sending.
+                            GridTaskWorker.this.jobRes.put(res.getJobId(), jobRes);
 
-            if (e instanceof Error)
-                throw (Error)e;
+                        return true;
+                    }
+                    // Catch Throwable to protect against bad user code.
+                    catch (Throwable e) {
+                        String errMsg = "Failed to failover job due to undeclared user exception [job=" +
+                            jobRes.getJob() + ", err=" + e + ']';
 
-            return false;
-        }
+                        U.error(log, errMsg, e);
+
+                        finishTask(null, new ComputeUserUndeclaredException(errMsg, e));
+
+                        if (e instanceof Error)
+                            throw (Error)e;
+
+                        return false;
+                    }
+                }
+            }
+        });
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/1d80a398/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
index 8530fbb..e4b6ece 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
@@ -50,9 +50,6 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
     private static final String CACHE_NAME = "myCache";
 
     /** */
-    private static final int MAX_FAILOVER_ATTEMPTS = 105;
-
-    /** */
     private static final int SERVERS_COUNT = 4;
 
     /** */
@@ -69,7 +66,6 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
         cfg.setDiscoverySpi(spi);
 
         AlwaysFailoverSpi failSpi = new AlwaysFailoverSpi();
-        failSpi.setMaximumFailoverAttempts(MAX_FAILOVER_ATTEMPTS);
         cfg.setFailoverSpi(failSpi);
 
         CacheConfiguration ccfg = defaultCacheConfiguration();


[38/50] ignite git commit: Fixed JavaDoc warning

Posted by vo...@apache.org.
Fixed JavaDoc warning


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

Branch: refs/heads/ignite-3341
Commit: 706a93593360c048b3f6504dc3be8ad620627b50
Parents: f260589
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Jun 16 13:38:40 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Jun 16 13:38:40 2016 +0300

----------------------------------------------------------------------
 .../cassandra/session/GenericBatchExecutionAssistant.java   | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/706a9359/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
index e80583a..633f723 100644
--- a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.cache.store.cassandra.session;
 
-import com.datastax.driver.core.Row;
 import java.util.HashSet;
 import java.util.Set;
+import com.datastax.driver.core.Row;
 
 /**
  * Implementation of the {@link org.apache.ignite.cache.store.cassandra.session.BatchExecutionAssistant}.
@@ -61,7 +61,12 @@ public abstract class GenericBatchExecutionAssistant<R, V> implements BatchExecu
         return false;
     }
 
-    /** {@inheritDoc} */
+    /**
+     * Processes the row.
+     *
+     * @param row Row.
+     */
     protected void process(Row row) {
+        // No-op.
     }
 }


[06/50] ignite git commit: IGNITE-3294: IGFS: Improved "create" performance in DUAL mode.

Posted by vo...@apache.org.
IGNITE-3294: IGFS: Improved "create" performance in DUAL mode.


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

Branch: refs/heads/ignite-3341
Commit: 40e6614f2426912aaec4220b8f720c12c3e5214a
Parents: 7f878c5
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Jun 14 11:09:01 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Jun 14 11:09:01 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsCreateResult.java       |  66 ++
 .../internal/processors/igfs/IgfsImpl.java      |  53 +-
 .../processors/igfs/IgfsMetaManager.java        | 612 +++++++++----------
 .../IgfsSecondaryFileSystemCreateContext.java   | 111 ++++
 .../IgfsSecondaryOutputStreamDescriptor.java    |  59 --
 .../meta/IgfsMetaDirectoryCreateProcessor.java  |  40 +-
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  |  46 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   4 +-
 8 files changed, 538 insertions(+), 453 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
new file mode 100644
index 0000000..0b09e02
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsCreateResult.java
@@ -0,0 +1,66 @@
+/*
+ * 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.igfs;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.OutputStream;
+
+/**
+ * IGFS file create result.
+ */
+public class IgfsCreateResult {
+    /** File info in the primary file system. */
+    private final IgfsEntryInfo info;
+
+    /** Output stream to the secondary file system. */
+    private final OutputStream secondaryOut;
+
+    /**
+     * Constructor.
+     *
+     * @param info File info in the primary file system.
+     * @param secondaryOut Output stream to the secondary file system.
+     */
+    public IgfsCreateResult(IgfsEntryInfo info, @Nullable OutputStream secondaryOut) {
+        assert info != null;
+
+        this.info = info;
+        this.secondaryOut = secondaryOut;
+    }
+
+    /**
+     * @return File info in the primary file system.
+     */
+    public IgfsEntryInfo info() {
+        return info;
+    }
+
+    /**
+     * @return Output stream to the secondary file system.
+     */
+    @Nullable public OutputStream secondaryOutputStream() {
+        return secondaryOut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsCreateResult.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 0808619..7234e52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -106,7 +106,6 @@ import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
-import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_WRITE;
 import static org.apache.ignite.events.EventType.EVT_IGFS_META_UPDATED;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_ASYNC;
 import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC;
@@ -1019,28 +1018,10 @@ public final class IgfsImpl implements IgfsEx {
                     log.debug("Open file for writing [path=" + path + ", bufSize=" + bufSize + ", overwrite=" +
                         overwrite + ", props=" + props + ']');
 
+                // Resolve mode.
                 final IgfsMode mode = resolveMode(path);
 
-                IgfsFileWorkerBatch batch;
-
-                if (mode != PRIMARY) {
-                    assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
-
-                    await(path);
-
-                    IgfsSecondaryOutputStreamDescriptor desc = meta.createDual(secondaryFs, path, simpleCreate,
-                        props, overwrite, bufSize, (short) replication, groupBlockSize(), affKey);
-
-                    batch = newBatch(path, desc.out());
-
-                    IgfsOutputStreamImpl os = new IgfsOutputStreamImpl(igfsCtx, path, desc.info(),
-                        bufferSize(bufSize), mode, batch);
-
-                    IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EVT_IGFS_FILE_OPENED_WRITE);
-
-                    return os;
-                }
-
+                // Prepare properties.
                 final Map<String, String> dirProps, fileProps;
 
                 if (props == null) {
@@ -1051,19 +1032,37 @@ public final class IgfsImpl implements IgfsEx {
                 else
                     dirProps = fileProps = new HashMap<>(props);
 
-                IgfsEntryInfo res = meta.create(
+                // Prepare context for DUAL mode.
+                IgfsSecondaryFileSystemCreateContext secondaryCtx = null;
+
+                if (mode != PRIMARY)
+                    secondaryCtx = new IgfsSecondaryFileSystemCreateContext(secondaryFs, path, overwrite, simpleCreate,
+                        fileProps, (short)replication, groupBlockSize(), bufSize);
+
+                // Await for async ops completion if in DUAL mode.
+                if (mode != PRIMARY)
+                    await(path);
+
+                // Perform create.
+                IgfsCreateResult res = meta.create(
                     path,
                     dirProps,
                     overwrite,
                     cfg.getBlockSize(),
                     affKey,
-                    evictExclude(path, true),
-                    fileProps
+                    evictExclude(path, mode == PRIMARY),
+                    fileProps,
+                    secondaryCtx
                 );
 
                 assert res != null;
 
-                return new IgfsOutputStreamImpl(igfsCtx, path, res, bufferSize(bufSize), mode, null);
+                // Create secondary file system batch.
+                OutputStream secondaryStream = res.secondaryOutputStream();
+
+                IgfsFileWorkerBatch batch = secondaryStream != null ? newBatch(path, secondaryStream) : null;
+
+                return new IgfsOutputStreamImpl(igfsCtx, path, res.info(), bufferSize(bufSize), mode, batch);
             }
         });
     }
@@ -1094,9 +1093,9 @@ public final class IgfsImpl implements IgfsEx {
 
                     await(path);
 
-                    IgfsSecondaryOutputStreamDescriptor desc = meta.appendDual(secondaryFs, path, bufSize);
+                    IgfsCreateResult desc = meta.appendDual(secondaryFs, path, bufSize);
 
-                    batch = newBatch(path, desc.out());
+                    batch = newBatch(path, desc.secondaryOutputStream());
 
                     return new IgfsOutputStreamImpl(igfsCtx, path, desc.info(), bufferSize(bufSize), mode, batch);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 2fb6066..c08d6a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -393,40 +393,47 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     public IgfsPathIds pathIds(IgfsPath path) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                validTxState(false);
+        // Prepare parts.
+        String[] components = path.componentsArray();
 
-                // Prepare parts.
-                String[] components = path.componentsArray();
+        String[] parts = new String[components.length + 1];
 
-                String[] parts = new String[components.length + 1];
+        System.arraycopy(components, 0, parts, 1, components.length);
 
-                System.arraycopy(components, 0, parts, 1, components.length);
+        // Get IDs.
+        if (client) {
+            List<IgniteUuid> ids = runClientTask(new IgfsClientMetaIdsForPathCallable(cfg.getName(), path));
 
-                // Prepare IDs.
-                IgniteUuid[] ids = new IgniteUuid[parts.length];
+            return new IgfsPathIds(path, parts, ids.toArray(new IgniteUuid[ids.size()]));
+        }
+        else {
+            if (busyLock.enterBusy()) {
+                try {
+                    validTxState(false);
 
-                ids[0] = IgfsUtils.ROOT_ID;
+                    IgniteUuid[] ids = new IgniteUuid[parts.length];
 
-                for (int i = 1; i < ids.length; i++) {
-                    IgniteUuid id = fileId(ids[i - 1], parts[i], false);
+                    ids[0] = IgfsUtils.ROOT_ID;
 
-                    if (id != null)
-                        ids[i] = id;
-                    else
-                        break;
-                }
+                    for (int i = 1; i < ids.length; i++) {
+                        IgniteUuid id = fileId(ids[i - 1], parts[i], false);
 
-                // Return.
-                return new IgfsPathIds(path, parts, ids);
-            }
-            finally {
-                busyLock.leaveBusy();
+                        if (id != null)
+                            ids[i] = id;
+                        else
+                            break;
+                    }
+
+                    // Return.
+                    return new IgfsPathIds(path, parts, ids);
+                }
+                finally {
+                    busyLock.leaveBusy();
+                }
             }
+            else
+                throw new IllegalStateException("Failed to get file IDS because Grid is stopping: " + path);
         }
-        else
-            throw new IllegalStateException("Failed to get file IDS because Grid is stopping: " + path);
     }
 
     /**
@@ -1146,6 +1153,46 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Wheter operation must be re-tries because we have suspicious links which may broke secondary file system
+     * consistency.
+     *
+     * @param pathIds Path IDs.
+     * @param lockInfos Lock infos.
+     * @return Whether to re-try.
+     */
+    private boolean isRetryForSecondary(IgfsPathIds pathIds, Map<IgniteUuid, IgfsEntryInfo> lockInfos) {
+        // We need to ensure that the last locked info is not linked with expected child.
+        // Otherwise there was some concurrent file system update and we have to re-try.
+        if (!pathIds.allExists()) {
+            // Find the last locked index
+            IgfsEntryInfo lastLockedInfo = null;
+            int lastLockedIdx = -1;
+
+            while (lastLockedIdx < pathIds.lastExistingIndex()) {
+                IgfsEntryInfo nextInfo = lockInfos.get(pathIds.id(lastLockedIdx + 1));
+
+                if (nextInfo != null) {
+                    lastLockedInfo = nextInfo;
+                    lastLockedIdx++;
+                }
+                else
+                    break;
+            }
+
+            assert lastLockedIdx < pathIds.count();
+
+            if (lastLockedInfo != null) {
+                String part = pathIds.part(lastLockedIdx + 1);
+
+                if (lastLockedInfo.listing().containsKey(part))
+                    return true;
+            }
+        }
+
+        return false;
+    }
+
+    /**
      * Move path to the trash directory.
      *
      * @param path Path.
@@ -1154,8 +1201,8 @@ public class IgfsMetaManager extends IgfsManager {
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
-    IgfsDeleteResult softDelete(final IgfsPath path, final boolean recursive, @Nullable IgfsSecondaryFileSystem secondaryFs)
-        throws IgniteCheckedException {
+    IgfsDeleteResult softDelete(final IgfsPath path, final boolean recursive,
+        @Nullable IgfsSecondaryFileSystem secondaryFs) throws IgniteCheckedException {
         while (true) {
             if (busyLock.enterBusy()) {
                 try {
@@ -1163,16 +1210,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                     IgfsPathIds pathIds = pathIds(path);
 
-                    boolean relaxed0 = relaxed;
-
-                    if (!pathIds.allExists()) {
-                        if (secondaryFs == null)
-                            // Return early if target path doesn't exist and we do not have secondary file system.
-                            return new IgfsDeleteResult(false, null);
-                        else
-                            // If path is missing partially, we need more serious locking for DUAL mode.
-                            relaxed0 = false;
-                    }
+                    if (!pathIds.allExists() && secondaryFs == null)
+                        return new IgfsDeleteResult(false, null);
 
                     IgniteUuid victimId = pathIds.lastId();
                     String victimName = pathIds.lastPart();
@@ -1183,7 +1222,7 @@ public class IgfsMetaManager extends IgfsManager {
                     // Prepare IDs to lock.
                     SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
-                    pathIds.addExistingIds(allIds, relaxed0);
+                    pathIds.addExistingIds(allIds, relaxed);
 
                     IgniteUuid trashId = IgfsUtils.randomTrashId();
 
@@ -1193,38 +1232,11 @@ public class IgfsMetaManager extends IgfsManager {
                         // Lock participants.
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(allIds);
 
-                        if (!pathIds.allExists()) {
-                            // We need to ensure that the last locked info is not linked with expected child.
-                            // Otherwise there was some ocncurrent file system update and we have to re-try.
-                            assert secondaryFs != null;
-
-                            // Find the last locked index
-                            IgfsEntryInfo lastLockedInfo = null;
-                            int lastLockedIdx = -1;
-
-                            while (lastLockedIdx < pathIds.lastExistingIndex()) {
-                                IgfsEntryInfo nextInfo = lockInfos.get(pathIds.id(lastLockedIdx + 1));
-
-                                if (nextInfo != null) {
-                                    lastLockedInfo = nextInfo;
-                                    lastLockedIdx++;
-                                }
-                                else
-                                    break;
-                            }
-
-                            assert lastLockedIdx < pathIds.count();
-
-                            if (lastLockedInfo != null) {
-                                String part = pathIds.part(lastLockedIdx + 1);
-
-                                if (lastLockedInfo.listing().containsKey(part))
-                                    continue;
-                            }
-                        }
+                        if (secondaryFs != null && isRetryForSecondary(pathIds, lockInfos))
+                            continue;
 
                         // Ensure that all participants are still in place.
-                        if (!pathIds.allExists() || !pathIds.verifyIntegrity(lockInfos, relaxed0)) {
+                        if (!pathIds.allExists() || !pathIds.verifyIntegrity(lockInfos, relaxed)) {
                             // For DUAL mode we will try to update the underlying FS still. Note we do that inside TX.
                             if (secondaryFs != null) {
                                 boolean res = secondaryFs.delete(path, recursive);
@@ -1717,7 +1729,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Transfer entry from one directory to another.
      *
-     * @param entry Entry to be transfered.
+     * @param entry Entry to be transferred.
      * @param srcId Source ID.
      * @param srcName Source name.
      * @param destId Destination ID.
@@ -1728,8 +1740,19 @@ public class IgfsMetaManager extends IgfsManager {
         IgniteUuid destId, String destName) throws IgniteCheckedException {
         validTxState(true);
 
-        id2InfoPrj.invoke(srcId, new IgfsMetaDirectoryListingRemoveProcessor(srcName, entry.fileId()));
-        id2InfoPrj.invoke(destId, new IgfsMetaDirectoryListingAddProcessor(destName, entry));
+        if (F.eq(srcId, destId)) {
+            id2InfoPrj.invoke(srcId, new IgfsMetaDirectoryListingRemoveProcessor(srcName, entry.fileId()));
+            id2InfoPrj.invoke(destId, new IgfsMetaDirectoryListingAddProcessor(destName, entry));
+        }
+        else {
+
+            Map<IgniteUuid, EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>> procMap = new HashMap<>();
+
+            procMap.put(srcId, new IgfsMetaDirectoryListingRemoveProcessor(srcName, entry.fileId()));
+            procMap.put(destId, new IgfsMetaDirectoryListingAddProcessor(destName, entry));
+
+            id2InfoPrj.invokeAll(procMap);
+        }
     }
 
     /**
@@ -1786,59 +1809,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Create the file in DUAL mode.
-     *
-     * @param fs File system.
-     * @param path Path.
-     * @param simpleCreate "Simple create" flag.
-     * @param props Properties..
-     * @param overwrite Overwrite flag.
-     * @param bufSize Buffer size.
-     * @param replication Replication factor.
-     * @param blockSize Block size.
-     * @param affKey Affinity key.
-     * @return Output stream descriptor.
-     * @throws IgniteCheckedException If file creation failed.
-     */
-    public IgfsSecondaryOutputStreamDescriptor createDual(final IgfsSecondaryFileSystem fs,
-        final IgfsPath path,
-        final boolean simpleCreate,
-        @Nullable final Map<String, String> props,
-        final boolean overwrite,
-        final int bufSize,
-        final short replication,
-        final long blockSize,
-        final IgniteUuid affKey)
-        throws IgniteCheckedException
-    {
-        if (busyLock.enterBusy()) {
-            try {
-                assert fs != null;
-                assert path != null;
-
-                // Events to fire (can be done outside of a transaction).
-                final Deque<IgfsEvent> pendingEvts = new LinkedList<>();
-
-                CreateFileSynchronizationTask task = new CreateFileSynchronizationTask(fs, path, simpleCreate, props,
-                    overwrite, bufSize, replication, blockSize, affKey, pendingEvts);
-
-                try {
-                    return synchronizeAndExecute(task, fs, false, path.parent());
-                }
-                finally {
-                    for (IgfsEvent evt : pendingEvts)
-                        evts.record(evt);
-                }
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to create file in DUAL mode because Grid is stopping: " + path);
-    }
-
-    /**
      * Append to a file in DUAL mode.
      *
      * @param fs File system.
@@ -1847,19 +1817,19 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Output stream descriptor.
      * @throws IgniteCheckedException If output stream open for append has failed.
      */
-    public IgfsSecondaryOutputStreamDescriptor appendDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
+    public IgfsCreateResult appendDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
         final int bufSize) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 assert fs != null;
                 assert path != null;
 
-                SynchronizationTask<IgfsSecondaryOutputStreamDescriptor> task =
-                    new SynchronizationTask<IgfsSecondaryOutputStreamDescriptor>() {
+                SynchronizationTask<IgfsCreateResult> task =
+                    new SynchronizationTask<IgfsCreateResult>() {
                         /** Output stream to the secondary file system. */
                         private OutputStream out;
 
-                        @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath,
+                        @Override public IgfsCreateResult onSuccess(Map<IgfsPath,
                             IgfsEntryInfo> infos) throws Exception {
                             validTxState(true);
 
@@ -1898,10 +1868,10 @@ public class IgfsMetaManager extends IgfsManager {
                             // Set lock and return.
                             IgfsEntryInfo lockedInfo = invokeLock(info.id(), false);
 
-                            return new IgfsSecondaryOutputStreamDescriptor(lockedInfo, out);
+                            return new IgfsCreateResult(lockedInfo, out);
                         }
 
-                        @Override public IgfsSecondaryOutputStreamDescriptor onFailure(@Nullable Exception err)
+                        @Override public IgfsCreateResult onFailure(@Nullable Exception err)
                             throws IgniteCheckedException {
                             U.closeQuiet(out);
 
@@ -2784,8 +2754,8 @@ public class IgfsMetaManager extends IgfsManager {
                         }
                         else {
                             // Create file and parent folders.
-                            IgfsPathsCreateResult res =
-                                createFile(pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+                            IgfsPathsCreateResult res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
+                                affKey, evictExclude, null);
 
                             if (res == null)
                                 continue;
@@ -2819,21 +2789,25 @@ public class IgfsMetaManager extends IgfsManager {
      * @param affKey Affinity key.
      * @param evictExclude Evict exclude flag.
      * @param fileProps File properties.
-     * @return @return Resulting info.
+     * @param secondaryCtx Secondary file system create context.
+     * @return @return Operation result.
      * @throws IgniteCheckedException If failed.
      */
-    IgfsEntryInfo create(
+    IgfsCreateResult create(
         final IgfsPath path,
         Map<String, String> dirProps,
         final boolean overwrite,
         final int blockSize,
         final @Nullable IgniteUuid affKey,
         final boolean evictExclude,
-        @Nullable Map<String, String> fileProps) throws IgniteCheckedException {
+        @Nullable Map<String, String> fileProps,
+        @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx) throws IgniteCheckedException {
         validTxState(false);
 
         while (true) {
             if (busyLock.enterBusy()) {
+                OutputStream secondaryOut = null;
+
                 try {
                     // Prepare path IDs.
                     IgfsPathIds pathIds = pathIds(path);
@@ -2860,6 +2834,9 @@ public class IgfsMetaManager extends IgfsManager {
                     try (IgniteInternalTx tx = startTx()) {
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
+                        if (secondaryCtx != null && isRetryForSecondary(pathIds, lockInfos))
+                            continue;
+
                         if (!pathIds.verifyIntegrity(lockInfos, relaxed))
                             // Directory structure changed concurrently. So we simply re-try.
                             continue;
@@ -2882,35 +2859,71 @@ public class IgfsMetaManager extends IgfsManager {
 
                             // At this point file can be re-created safely.
 
-                            // First step: add existing to trash listing.
+                            // Add existing to trash listing.
                             IgniteUuid oldId = pathIds.lastId();
 
                             id2InfoPrj.invoke(trashId, new IgfsMetaDirectoryListingAddProcessor(oldId.toString(),
                                 new IgfsListingEntry(oldInfo)));
 
-                            // Second step: replace ID in parent directory.
+                            // Replace ID in parent directory.
                             String name = pathIds.lastPart();
                             IgniteUuid parentId = pathIds.lastParentId();
 
                             id2InfoPrj.invoke(parentId, new IgfsMetaDirectoryListingReplaceProcessor(name, overwriteId));
 
-                            // Third step: create the file.
-                            long createTime = System.currentTimeMillis();
+                            // Create the file.
+                            IgniteUuid newLockId = createFileLockId(false);
+
+                            long newAccessTime;
+                            long newModificationTime;
+                            Map<String, String> newProps;
+                            long newLen;
+                            int newBlockSize;
+
+                            if (secondaryCtx != null) {
+                                secondaryOut = secondaryCtx.create();
 
-                            IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new IgfsMetaFileCreateProcessor(createTime,
-                                fileProps, blockSize, affKey, createFileLockId(false), evictExclude));
+                                IgfsFile secondaryFile = secondaryCtx.info();
+
+                                if (secondaryFile == null)
+                                    throw fsException("Failed to open output stream to the file created in " +
+                                        "the secondary file system because it no longer exists: " + path);
+                                else if (secondaryFile.isDirectory())
+                                    throw fsException("Failed to open output stream to the file created in " +
+                                        "the secondary file system because the path points to a directory: " + path);
+
+                                newAccessTime = secondaryFile.accessTime();
+                                newModificationTime = secondaryFile.modificationTime();
+                                newProps = secondaryFile.properties();
+                                newLen = secondaryFile.length();
+                                newBlockSize = secondaryFile.blockSize();
+                            }
+                            else {
+                                newAccessTime = System.currentTimeMillis();
+                                newModificationTime = newAccessTime;
+                                newProps = fileProps;
+                                newLen = 0L;
+                                newBlockSize = blockSize;
+                            }
+
+                            IgfsEntryInfo newInfo = invokeAndGet(overwriteId,
+                                new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps,
+                                    newBlockSize, affKey, newLockId, evictExclude, newLen));
 
                             // Prepare result and commit.
                             tx.commit();
 
                             IgfsUtils.sendEvents(igfsCtx.kernalContext(), path, EventType.EVT_IGFS_FILE_OPENED_WRITE);
 
-                            return newInfo;
+                            return new IgfsCreateResult(newInfo, secondaryOut);
                         }
                         else {
                             // Create file and parent folders.
-                            IgfsPathsCreateResult res =
-                                createFile(pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+                            if (secondaryCtx != null)
+                                secondaryOut = secondaryCtx.create();
+
+                            IgfsPathsCreateResult res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
+                                affKey, evictExclude, secondaryCtx);
 
                             if (res == null)
                                 continue;
@@ -2921,10 +2934,20 @@ public class IgfsMetaManager extends IgfsManager {
                             // Generate events.
                             generateCreateEvents(res.createdPaths(), true);
 
-                            return res.info();
+                            return new IgfsCreateResult(res.info(), secondaryOut);
                         }
                     }
                 }
+                catch (IgniteException | IgniteCheckedException e) {
+                    U.closeQuiet(secondaryOut);
+
+                    throw e;
+                }
+                catch (Exception e) {
+                    U.closeQuiet(secondaryOut);
+
+                    throw new IgniteCheckedException("Create failed due to unexpected exception: " + path, e);
+                }
                 finally {
                     busyLock.leaveBusy();
                 }
@@ -2950,7 +2973,7 @@ public class IgfsMetaManager extends IgfsManager {
             throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
                 "element is not a directory)");
 
-        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false);
+        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false, null);
     }
 
     /**
@@ -2963,22 +2986,25 @@ public class IgfsMetaManager extends IgfsManager {
      * @param blockSize Block size.
      * @param affKey Affinity key (optional)
      * @param evictExclude Evict exclude flag.
+     * @param secondaryCtx Secondary file system create context.
      * @return Result or {@code} if the first parent already contained child with the same name.
      * @throws IgniteCheckedException If failed.
      */
     @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map<IgniteUuid, IgfsEntryInfo> lockInfos,
         Map<String, String> dirProps, Map<String, String> fileProps, int blockSize, @Nullable IgniteUuid affKey,
-        boolean evictExclude) throws IgniteCheckedException{
+        boolean evictExclude, @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx)
+        throws IgniteCheckedException{
         // Check if entry we are going to write to is directory.
         if (lockInfos.get(pathIds.lastExistingId()).isFile())
             throw new IgfsParentNotDirectoryException("Failed to open file for write " +
                 "(parent element is not a directory): " + pathIds.path());
 
-        return createFileOrDirectory(false, pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+        return createFileOrDirectory(false, pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude,
+            secondaryCtx);
     }
 
     /**
-     * Ceate file or directory.
+     * Create file or directory.
      *
      * @param dir Directory flag.
      * @param pathIds Path IDs.
@@ -2988,12 +3014,15 @@ public class IgfsMetaManager extends IgfsManager {
      * @param blockSize Block size.
      * @param affKey Affinity key.
      * @param evictExclude Evict exclude flag.
+     * @param secondaryCtx Secondary file system create context.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
+    @SuppressWarnings("unchecked")
     private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pathIds,
         Map<IgniteUuid, IgfsEntryInfo> lockInfos, Map<String, String> dirProps, Map<String, String> fileProps,
-        int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude) throws IgniteCheckedException {
+        int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude,
+        @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx) throws IgniteCheckedException {
         // This is our starting point.
         int lastExistingIdx = pathIds.lastExistingIndex();
         IgfsEntryInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId());
@@ -3009,8 +3038,10 @@ public class IgfsMetaManager extends IgfsManager {
         if (lastExistingInfo.hasChild(curPart))
             return null;
 
+        Map<IgniteUuid, EntryProcessor> procMap = new HashMap<>();
+
         // First step: add new entry to the last existing element.
-        id2InfoPrj.invoke(lastExistingInfo.id(), new IgfsMetaDirectoryListingAddProcessor(curPart,
+        procMap.put(lastExistingInfo.id(), new IgfsMetaDirectoryListingAddProcessor(curPart,
             new IgfsListingEntry(curId, dir || !pathIds.isLastIndex(curIdx))));
 
         // Events support.
@@ -3019,20 +3050,44 @@ public class IgfsMetaManager extends IgfsManager {
         List<IgfsPath> createdPaths = new ArrayList<>(pathIds.count() - curIdx);
 
         // Second step: create middle directories.
-        long createTime = System.currentTimeMillis();
+        long curTime = System.currentTimeMillis();
 
         while (curIdx < pathIds.count() - 1) {
+            lastCreatedPath = new IgfsPath(lastCreatedPath, curPart);
+
             int nextIdx = curIdx + 1;
 
             String nextPart = pathIds.part(nextIdx);
             IgniteUuid nextId = pathIds.surrogateId(nextIdx);
 
-            id2InfoPrj.invoke(curId, new IgfsMetaDirectoryCreateProcessor(createTime, dirProps,
+            long accessTime;
+            long modificationTime;
+            Map<String, String> props;
+
+            if (secondaryCtx != null) {
+                IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(lastCreatedPath);
+
+                if (secondaryInfo == null)
+                    throw new IgfsException("Failed to perform operation because secondary file system path was " +
+                        "modified concurrnetly: " + lastCreatedPath);
+                else if (secondaryInfo.isFile())
+                    throw new IgfsException("Failed to perform operation because secondary file system entity is " +
+                        "not directory: " + lastCreatedPath);
+
+                accessTime = secondaryInfo.accessTime();
+                modificationTime = secondaryInfo.modificationTime();
+                props = secondaryInfo.properties();
+            }
+            else {
+                accessTime = curTime;
+                modificationTime = curTime;
+                props = dirProps;
+            }
+
+            procMap.put(curId, new IgfsMetaDirectoryCreateProcessor(accessTime, modificationTime, props,
                 nextPart, new IgfsListingEntry(nextId, dir || !pathIds.isLastIndex(nextIdx))));
 
             // Save event.
-            lastCreatedPath = new IgfsPath(lastCreatedPath, curPart);
-
             createdPaths.add(lastCreatedPath);
 
             // Advance things further.
@@ -3043,16 +3098,75 @@ public class IgfsMetaManager extends IgfsManager {
         }
 
         // Third step: create leaf.
-        IgfsEntryInfo info;
+        if (dir) {
+            long accessTime;
+            long modificationTime;
+            Map<String, String> props;
 
-        if (dir)
-            info = invokeAndGet(curId, new IgfsMetaDirectoryCreateProcessor(createTime, dirProps));
-        else
-            info = invokeAndGet(curId, new IgfsMetaFileCreateProcessor(createTime, fileProps,
-                blockSize, affKey, createFileLockId(false), evictExclude));
+            if (secondaryCtx != null) {
+                IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(pathIds.path());
+
+                if (secondaryInfo == null)
+                    throw new IgfsException("Failed to perform operation because secondary file system path was " +
+                        "modified concurrnetly: " + pathIds.path());
+                else if (secondaryInfo.isFile())
+                    throw new IgfsException("Failed to perform operation because secondary file system entity is " +
+                        "not directory: " + lastCreatedPath);
+
+                accessTime = secondaryInfo.accessTime();
+                modificationTime = secondaryInfo.modificationTime();
+                props = secondaryInfo.properties();
+            }
+            else {
+                accessTime = curTime;
+                modificationTime = curTime;
+                props = dirProps;
+            }
+
+            procMap.put(curId, new IgfsMetaDirectoryCreateProcessor(accessTime, modificationTime, props));
+        }
+        else {
+            long newAccessTime;
+            long newModificationTime;
+            Map<String, String> newProps;
+            long newLen;
+            int newBlockSize;
+
+            if (secondaryCtx != null) {
+                IgfsFile secondaryFile = secondaryCtx.info();
+
+                if (secondaryFile == null)
+                    throw fsException("Failed to open output stream to the file created in " +
+                        "the secondary file system because it no longer exists: " + pathIds.path());
+                else if (secondaryFile.isDirectory())
+                    throw fsException("Failed to open output stream to the file created in " +
+                        "the secondary file system because the path points to a directory: " + pathIds.path());
+
+                newAccessTime = secondaryFile.accessTime();
+                newModificationTime = secondaryFile.modificationTime();
+                newProps = secondaryFile.properties();
+                newLen = secondaryFile.length();
+                newBlockSize = secondaryFile.blockSize();
+            }
+            else {
+                newAccessTime = curTime;
+                newModificationTime = curTime;
+                newProps = fileProps;
+                newLen = 0L;
+                newBlockSize = blockSize;
+            }
+
+            procMap.put(curId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps,
+                newBlockSize, affKey, createFileLockId(false), evictExclude, newLen));
+        }
 
         createdPaths.add(pathIds.path());
 
+        // Execute cache operations.
+        Map<Object, EntryProcessorResult> invokeRes = ((IgniteInternalCache)id2InfoPrj).invokeAll(procMap);
+
+        IgfsEntryInfo info = (IgfsEntryInfo)invokeRes.get(curId).get();
+
         return new IgfsPathsCreateResult(createdPaths, info);
     }
 
@@ -3111,182 +3225,4 @@ public class IgfsMetaManager extends IgfsManager {
          */
         public T onFailure(Exception err) throws IgniteCheckedException;
     }
-
-    /**
-     * Synchronization task to create a file.
-     */
-    private class CreateFileSynchronizationTask implements SynchronizationTask<IgfsSecondaryOutputStreamDescriptor> {
-        /** Secondary file system. */
-        private IgfsSecondaryFileSystem fs;
-
-        /** Path. */
-        private IgfsPath path;
-
-        /** Simple create flag. */
-        private boolean simpleCreate;
-
-        /** Properties. */
-        private Map<String, String> props;
-
-        /** Overwrite flag. */
-        private boolean overwrite;
-
-        /** Buffer size. */
-        private int bufSize;
-
-        /** Replication factor. */
-        private short replication;
-
-        /** Block size. */
-        private long blockSize;
-
-        /** Affinity key. */
-        private IgniteUuid affKey;
-
-        /** Pending events. */
-        private Deque<IgfsEvent> pendingEvts;
-
-        /** Output stream to the secondary file system. */
-        private OutputStream out;
-
-        /**
-         * Constructor.
-         *
-         * @param fs Secondary file system.
-         * @param path Path.
-         * @param simpleCreate Simple create flag.
-         * @param props Properties.
-         * @param overwrite Overwrite flag.
-         * @param bufSize Buffer size.
-         * @param replication Replication factor.
-         * @param blockSize Block size.
-         * @param affKey Affinity key.
-         * @param pendingEvts Pending events.
-         */
-        public CreateFileSynchronizationTask(IgfsSecondaryFileSystem fs, IgfsPath path, boolean simpleCreate,
-            @Nullable Map<String, String> props, boolean overwrite, int bufSize, short replication, long blockSize,
-            IgniteUuid affKey, Deque<IgfsEvent> pendingEvts) {
-            this.fs = fs;
-            this.path = path;
-            this.simpleCreate = simpleCreate;
-            this.props = props;
-            this.overwrite = overwrite;
-            this.bufSize = bufSize;
-            this.replication = replication;
-            this.blockSize = blockSize;
-            this.affKey = affKey;
-            this.pendingEvts = pendingEvts;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map<IgfsPath, IgfsEntryInfo> infos)
-            throws Exception {
-            validTxState(true);
-
-            assert !infos.isEmpty();
-
-            // Determine the first existing parent.
-            IgfsPath parentPath = null;
-
-            for (IgfsPath curPath : infos.keySet()) {
-                if (parentPath == null || curPath.isSubDirectoryOf(parentPath))
-                    parentPath = curPath;
-            }
-
-            assert parentPath != null;
-
-            IgfsEntryInfo parentInfo = infos.get(parentPath);
-
-            // Delegate to the secondary file system.
-            out = simpleCreate ? fs.create(path, overwrite) :
-                fs.create(path, bufSize, overwrite, replication, blockSize, props);
-
-            IgfsPath parent0 = path.parent();
-
-            assert parent0 != null : "path.parent() is null (are we creating ROOT?): " + path;
-
-            // If some of the parent directories were missing, synchronize again.
-            if (!parentPath.equals(parent0)) {
-                parentInfo = synchronize(fs, parentPath, parentInfo, parent0, true, null);
-
-                // Fire notification about missing directories creation.
-                if (evts.isRecordable(EventType.EVT_IGFS_DIR_CREATED)) {
-                    IgfsPath evtPath = parent0;
-
-                    while (!parentPath.equals(evtPath)) {
-                        pendingEvts.addFirst(new IgfsEvent(evtPath, locNode,
-                            EventType.EVT_IGFS_DIR_CREATED));
-
-                        evtPath = evtPath.parent();
-
-                        assert evtPath != null; // If this fails, then ROOT does not exist.
-                    }
-                }
-            }
-
-            // Get created file info.
-            IgfsFile status = fs.info(path);
-
-            if (status == null)
-                throw fsException("Failed to open output stream to the file created in " +
-                    "the secondary file system because it no longer exists: " + path);
-            else if (status.isDirectory())
-                throw fsException("Failed to open output stream to the file created in " +
-                    "the secondary file system because the path points to a directory: " + path);
-
-            IgfsEntryInfo newInfo = IgfsUtils.createFile(
-                IgniteUuid.randomUuid(),
-                status.blockSize(),
-                status.length(),
-                affKey,
-                createFileLockId(false),
-                igfsCtx.igfs().evictExclude(path, false),
-                status.properties(),
-                status.accessTime(),
-                status.modificationTime()
-            );
-
-            // Add new file info to the listing optionally removing the previous one.
-            assert parentInfo != null;
-
-            IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), path.name(), newInfo);
-
-            if (oldId != null) {
-                IgfsEntryInfo oldInfo = info(oldId);
-
-                assert oldInfo != null; // Otherwise cache is in inconsistent state.
-
-                // The contact is that we cannot overwrite a file locked for writing:
-                if (oldInfo.lockId() != null)
-                    throw fsException("Failed to overwrite file (file is opened for writing) [path=" +
-                        path + ", fileId=" + oldId + ", lockId=" + oldInfo.lockId() + ']');
-
-                id2InfoPrj.remove(oldId); // Remove the old one.
-                id2InfoPrj.invoke(parentInfo.id(), new IgfsMetaDirectoryListingRemoveProcessor(
-                    path.name(), parentInfo.listing().get(path.name()).fileId()));
-
-                createNewEntry(newInfo, parentInfo.id(), path.name()); // Put new one.
-
-                igfsCtx.data().delete(oldInfo);
-            }
-
-            // Record CREATE event if needed.
-            if (oldId == null && evts.isRecordable(EventType.EVT_IGFS_FILE_CREATED))
-                pendingEvts.add(new IgfsEvent(path, locNode, EventType.EVT_IGFS_FILE_CREATED));
-
-            return new IgfsSecondaryOutputStreamDescriptor(newInfo, out);
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsSecondaryOutputStreamDescriptor onFailure(Exception err) throws IgniteCheckedException {
-            U.closeQuiet(out);
-
-            U.error(log, "File create in DUAL mode failed [path=" + path + ", simpleCreate=" +
-                simpleCreate + ", props=" + props + ", overwrite=" + overwrite + ", bufferSize=" +
-                bufSize + ", replication=" + replication + ", blockSize=" + blockSize + ']', err);
-
-            throw new IgniteCheckedException("Failed to create the file due to secondary file system " +
-                "exception: " + path, err);
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
new file mode 100644
index 0000000..1c0efd6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
@@ -0,0 +1,111 @@
+/*
+ * 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.igfs;
+
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.OutputStream;
+import java.util.Map;
+
+/**
+ * Context for secondary file system create request.
+ */
+public class IgfsSecondaryFileSystemCreateContext {
+    /** File system. */
+    private final IgfsSecondaryFileSystem fs;
+
+    /** Path. */
+    private final IgfsPath path;
+
+    /** Overwrite flag. */
+    private final boolean overwrite;
+
+    /** Simple create flag. */
+    private final boolean simpleCreate;
+
+    /** Properties. */
+    private final Map<String, String> props;
+
+    /** Replication. */
+    private final short replication;
+
+    /** Block size. */
+    private final long blockSize;
+
+    /** Buffer size. */
+    private final int bufSize;
+
+    /**
+     * Constructor.
+     *
+     * @param fs File system.
+     * @param path Path.
+     * @param overwrite Overwrite flag.
+     * @param simpleCreate Simple create flag.
+     * @param props Properties.
+     * @param replication Replication.
+     * @param blockSize Block size.
+     * @param bufSize Buffer size.
+     */
+    public IgfsSecondaryFileSystemCreateContext(IgfsSecondaryFileSystem fs, IgfsPath path, boolean overwrite,
+        boolean simpleCreate, @Nullable Map<String, String> props, short replication, long blockSize, int bufSize) {
+        this.fs = fs;
+        this.path = path;
+        this.overwrite = overwrite;
+        this.simpleCreate = simpleCreate;
+        this.props = props;
+        this.replication = replication;
+        this.blockSize = blockSize;
+        this.bufSize = bufSize;
+    }
+
+    /**
+     * Create file in the secondary file system.
+     *
+     * @return Output stream.
+     */
+    public OutputStream create() {
+        return simpleCreate ? fs.create(path, overwrite) :
+            fs.create(path, bufSize, overwrite, replication, blockSize, props);
+    }
+
+    /**
+     * Get file info.
+     *
+     * @return File.
+     */
+    public IgfsFile info() {
+        return fs.info(path);
+    }
+
+    /**
+     * @return Secondary file system.
+     */
+    public IgfsSecondaryFileSystem fileSystem() {
+        return fs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsSecondaryFileSystemCreateContext.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
deleted file mode 100644
index 6bbc2c0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java
+++ /dev/null
@@ -1,59 +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.igfs;
-
-import java.io.OutputStream;
-
-/**
- * Descriptor of an output stream opened to the secondary file system.
- */
-public class IgfsSecondaryOutputStreamDescriptor {
-    /** File info in the primary file system. */
-    private final IgfsEntryInfo info;
-
-    /** Output stream to the secondary file system. */
-    private final OutputStream out;
-
-    /**
-     * Constructor.
-     *
-     * @param info File info in the primary file system.
-     * @param out Output stream to the secondary file system.
-     */
-    IgfsSecondaryOutputStreamDescriptor(IgfsEntryInfo info, OutputStream out) {
-        assert info != null;
-        assert out != null;
-
-        this.info = info;
-        this.out = out;
-    }
-
-    /**
-     * @return File info in the primary file system.
-     */
-    IgfsEntryInfo info() {
-        return info;
-    }
-
-    /**
-     * @return Output stream to the secondary file system.
-     */
-    OutputStream out() {
-        return out;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
index eee9300..b016633 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
@@ -48,8 +48,11 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Create time. */
-    private long createTime;
+    /** Access time. */
+    private long accessTime;
+
+    /** Modification time. */
+    private long modificationTime;
 
     /** Properties. */
     private Map<String, String> props;
@@ -70,24 +73,27 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     /**
      * Constructor.
      *
-     * @param createTime Create time.
+     * @param accessTime Create time.
+     * @param modificationTime Modification time.
      * @param props Properties.
      */
-    public IgfsMetaDirectoryCreateProcessor(long createTime, Map<String, String> props) {
-        this(createTime, props, null, null);
+    public IgfsMetaDirectoryCreateProcessor(long accessTime, long modificationTime, Map<String, String> props) {
+        this(accessTime, modificationTime, props, null, null);
     }
 
     /**
      * Constructor.
      *
-     * @param createTime Create time.
+     * @param accessTime Create time.
+     * @param modificationTime Modification time.
      * @param props Properties.
      * @param childName Child name.
      * @param childEntry Child entry.
      */
-    public IgfsMetaDirectoryCreateProcessor(long createTime, Map<String, String> props, String childName,
-        IgfsListingEntry childEntry) {
-        this.createTime = createTime;
+    public IgfsMetaDirectoryCreateProcessor(long accessTime, long modificationTime, Map<String, String> props,
+        String childName, IgfsListingEntry childEntry) {
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
         this.props = props;
         this.childName = childName;
         this.childEntry = childEntry;
@@ -101,8 +107,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
             entry.getKey(),
             null,
             props,
-            createTime,
-            createTime
+            accessTime,
+            modificationTime
         );
 
         if (childName != null)
@@ -115,7 +121,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
 
@@ -127,7 +134,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        createTime = in.readLong();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
 
@@ -141,7 +149,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
         BinaryRawWriter out = writer.rawWriter();
 
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
 
@@ -155,7 +164,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
         BinaryRawReader in = reader.rawReader();
 
-        createTime = in.readLong();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
index 8c4c296..a3e9d48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
@@ -49,7 +49,10 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     private static final long serialVersionUID = 0L;
 
     /** Create time. */
-    private long createTime;
+    private long accessTime;
+
+    /** Modification time. */
+    private long modificationTime;
 
     /** Properties. */
     private Map<String, String> props;
@@ -66,6 +69,9 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     /** Evict exclude flag. */
     private boolean evictExclude;
 
+    /** File length. */
+    private long len;
+
     /**
      * Constructor.
      */
@@ -76,21 +82,25 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     /**
      * Constructor.
      *
-     * @param createTime Create time.
+     * @param accessTime Access time.
+     * @param modificationTime Modification time.
      * @param props Properties.
      * @param blockSize Block size.
      * @param affKey Affinity key.
      * @param lockId Lock ID.
      * @param evictExclude Evict exclude flag.
+     * @param len File length.
      */
-    public IgfsMetaFileCreateProcessor(long createTime, Map<String, String> props, int blockSize,
-        @Nullable IgniteUuid affKey, IgniteUuid lockId, boolean evictExclude) {
-        this.createTime = createTime;
+    public IgfsMetaFileCreateProcessor(long accessTime, long modificationTime, Map<String, String> props,
+        int blockSize, @Nullable IgniteUuid affKey, IgniteUuid lockId, boolean evictExclude, long len) {
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
         this.props = props;
         this.blockSize = blockSize;
         this.affKey = affKey;
         this.lockId = lockId;
         this.evictExclude = evictExclude;
+        this.len = len;
     }
 
     /** {@inheritDoc} */
@@ -99,13 +109,13 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
         IgfsEntryInfo info = IgfsUtils.createFile(
             entry.getKey(),
             blockSize,
-            0L,
+            len,
             affKey,
             lockId,
             evictExclude,
             props,
-            createTime,
-            createTime
+            accessTime,
+            modificationTime
         );
 
         entry.setValue(info);
@@ -115,7 +125,8 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
 
@@ -123,11 +134,14 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
         U.writeGridUuid(out, affKey);
         U.writeGridUuid(out, lockId);
         out.writeBoolean(evictExclude);
+
+        out.writeLong(len);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        createTime = in.readLong();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
 
@@ -135,13 +149,16 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
         affKey = U.readGridUuid(in);
         lockId = U.readGridUuid(in);
         evictExclude = in.readBoolean();
+
+        len = in.readLong();
     }
 
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
         BinaryRawWriter out = writer.rawWriter();
 
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
 
@@ -149,13 +166,16 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
         BinaryUtils.writeIgniteUuid(out, affKey);
         BinaryUtils.writeIgniteUuid(out, lockId);
         out.writeBoolean(evictExclude);
+
+        out.writeLong(len);
     }
 
     /** {@inheritDoc} */
     @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
         BinaryRawReader in = reader.rawReader();
 
-        createTime = in.readLong();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
 
@@ -163,6 +183,8 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
         affKey = BinaryUtils.readIgniteUuid(in);
         lockId = BinaryUtils.readIgniteUuid(in);
         evictExclude = in.readBoolean();
+
+        len = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/40e6614f/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index 8b88157..6053d3b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -142,7 +142,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         assertEmpty(mgr.directoryListing(ROOT_ID));
 
         assertTrue(mgr.mkdirs(new IgfsPath("/dir"), IgfsImpl.DFLT_DIR_META));
-        assertNotNull(mgr.create(new IgfsPath("/file"), null, false, 400, null, false, null));
+        assertNotNull(mgr.create(new IgfsPath("/file"), null, false, 400, null, false, null, null));
 
         IgfsListingEntry dirEntry = mgr.directoryListing(ROOT_ID).get("dir");
         assertNotNull(dirEntry);
@@ -214,7 +214,7 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
     private IgfsEntryInfo createFileAndGetInfo(String path) throws IgniteCheckedException {
         IgfsPath p = path(path);
 
-        IgfsEntryInfo res = mgr.create(p, null, false, 400, null, false, null);
+        IgfsEntryInfo res = mgr.create(p, null, false, 400, null, false, null, null).info();
 
         assert res != null;
         assert !res.isDirectory();


[31/50] ignite git commit: IGNITE-2767: Cont. query remote filter requires to be in client nodes class path. Reviewed and merged by Denis Magda.

Posted by vo...@apache.org.
IGNITE-2767: Cont. query remote filter requires to be in client nodes class path.
Reviewed and merged by Denis Magda.


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

Branch: refs/heads/ignite-3341
Commit: 377b387eff8d678f85f9d49c1a351e72cccfe105
Parents: 1cb291c
Author: Vladislav Pyatkov <vl...@gmail.com>
Authored: Wed Jun 15 17:18:27 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jun 15 17:18:27 2016 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   7 +-
 ...yRemoteFilterMissingInClassPathSelfTest.java | 237 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite3.java         |   2 +
 3 files changed, 245 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/377b387e/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 73b541f..308830e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -51,6 +51,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.AddressResolver;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -1713,7 +1714,11 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
                 data0.put(entry.getKey(), compData);
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to unmarshal discovery data for component: "  + entry.getKey(), e);
+                if (GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC.ordinal() == entry.getKey() &&
+                    X.hasCause(e, ClassNotFoundException.class) && locNode.isClient())
+                    U.warn(log, "Failed to unmarshal continuous query remote filter on client node. Can be ignored.");
+                else
+                    U.error(log, "Failed to unmarshal discovery data for component: "  + entry.getKey(), e);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/377b387e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
new file mode 100644
index 0000000..37f4e01
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/ContinuousQueryRemoteFilterMissingInClassPathSelfTest.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridStringLogger;
+import org.apache.ignite.testframework.config.GridTestProperties;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import javax.cache.configuration.Factory;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryEventFilter;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+/**
+ *
+ */
+public class ContinuousQueryRemoteFilterMissingInClassPathSelfTest extends GridCommonAbstractTest {
+    /** URL of classes. */
+    private static final URL[] URLS;
+
+    static {
+        try {
+            URLS = new URL[] {new URL(GridTestProperties.getProperty("p2p.uri.cls"))};
+        }
+        catch (MalformedURLException e) {
+            throw new RuntimeException("Define property p2p.uri.cls", e);
+        }
+    }
+
+    /** */
+    private GridStringLogger log;
+
+    /** */
+    private boolean clientMode;
+
+    /** */
+    private boolean setExternalLoader;
+
+    /** */
+    private ClassLoader ldr;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(name);
+
+        cfg.setClientMode(clientMode);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setName("simple");
+
+        cacheCfg.setCacheMode(CacheMode.PARTITIONED);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        if (setExternalLoader)
+            cfg.setClassLoader(ldr);
+        else
+            cfg.setGridLogger(log);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If fail.
+     */
+    public void testWarningMessageOnClientNode() throws Exception {
+        ldr = new URLClassLoader(URLS, getClass().getClassLoader());
+
+        clientMode = false;
+        setExternalLoader = true;
+        final Ignite ignite0 = startGrid(1);
+
+        executeContiniouseQuery(ignite0.cache("simple"));
+
+        log = new GridStringLogger();
+        clientMode = true;
+        setExternalLoader = false;
+
+        startGrid(2);
+
+        assertTrue(log.toString().contains("Failed to unmarshal continuous query remote filter on client node. " +
+            "Can be ignored."));
+    }
+
+    /**
+     * @throws Exception If fail.
+     */
+    public void testNoWarningMessageOnClientNode() throws Exception {
+        ldr = new URLClassLoader(URLS, getClass().getClassLoader());
+
+        setExternalLoader = true;
+
+        clientMode = false;
+        final Ignite ignite0 = startGrid(1);
+
+        executeContiniouseQuery(ignite0.cache("simple"));
+
+        log = new GridStringLogger();
+        clientMode = true;
+
+        startGrid(2);
+
+        assertTrue(!log.toString().contains("Failed to unmarshal continuous query remote filter on client node. " +
+            "Can be ignored."));
+    }
+
+    /**
+     * @throws Exception If fail.
+     */
+    public void testExceptionOnServerNode() throws Exception {
+        ldr = new URLClassLoader(URLS, getClass().getClassLoader());
+
+        clientMode = false;
+
+        setExternalLoader = true;
+        final Ignite ignite0 = startGrid(1);
+
+        executeContiniouseQuery(ignite0.cache("simple"));
+
+        log = new GridStringLogger();
+        setExternalLoader = false;
+
+        startGrid(2);
+
+        assertTrue(log.toString().contains("class org.apache.ignite.IgniteCheckedException: " +
+            "Failed to find class with given class loader for unmarshalling"));
+    }
+
+    /**
+     * @throws Exception If fail.
+     */
+    public void testNoExceptionOnServerNode() throws Exception {
+        ldr = new URLClassLoader(URLS, getClass().getClassLoader());
+
+        clientMode = false;
+
+        setExternalLoader = true;
+        final Ignite ignite0 = startGrid(1);
+
+        executeContiniouseQuery(ignite0.cache("simple"));
+
+        log = new GridStringLogger();
+
+        startGrid(2);
+
+        assertTrue(!log.toString().contains("class org.apache.ignite.IgniteCheckedException: " +
+            "Failed to find class with given class loader for unmarshalling"));
+    }
+
+    /**
+     * @param cache Ignite cache.
+     * @throws Exception If fail.
+     */
+    private void executeContiniouseQuery(IgniteCache cache) throws Exception {
+        ContinuousQuery<Integer, String> qry = new ContinuousQuery<>();
+
+        qry.setLocalListener(
+            new CacheEntryUpdatedListener<Integer, String>() {
+                @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends String>> events)
+                    throws CacheEntryListenerException {
+                    for (CacheEntryEvent<? extends Integer, ? extends String> event : events)
+                        System.out.println("Key = " + event.getKey() + ", Value = " + event.getValue());
+                }
+            }
+        );
+
+        final Class<CacheEntryEventSerializableFilter> remoteFilterClass = (Class<CacheEntryEventSerializableFilter>)
+            ldr.loadClass("org.apache.ignite.tests.p2p.CacheDeploymentCacheEntryEventSerializableFilter");
+
+        qry.setRemoteFilterFactory(new ClassFilterFactory(remoteFilterClass));
+
+        cache.query(qry);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, "Message " + i);
+    }
+
+    /**
+     *
+     */
+    private static class ClassFilterFactory implements Factory<CacheEntryEventFilter<Integer, String>> {
+        /** */
+        private Class<CacheEntryEventSerializableFilter> cls;
+
+        /**
+         * @param cls Class.
+         */
+        public ClassFilterFactory(Class<CacheEntryEventSerializableFilter> cls) {
+            this.cls = cls;
+        }
+
+        /** {@inheritDoc} */
+        @Override public CacheEntryEventSerializableFilter<Integer, String> create() {
+            try {
+                return cls.newInstance();
+            }
+            catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/377b387e/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
index dbef1fb..7ad7ad5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBin
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationStoreEnabledTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheKeepBinaryIterationSwapEnabledTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.ContinuousQueryRemoteFilterMissingInClassPathSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapTieredTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapValuesTest;
@@ -112,6 +113,7 @@ public class IgniteCacheQuerySelfTestSuite3 extends TestSuite {
         suite.addTestSuite(CacheContinuousBatchForceServerModeAckTest.class);
         suite.addTestSuite(CacheContinuousQueryExecuteInPrimaryTest.class);
         suite.addTestSuite(CacheContinuousQueryLostPartitionTest.class);
+        suite.addTestSuite(ContinuousQueryRemoteFilterMissingInClassPathSelfTest.class);
         suite.addTestSuite(IgniteCacheContinuousQueryImmutableEntryTest.class);
         suite.addTestSuite(CacheKeepBinaryIterationTest.class);
         suite.addTestSuite(CacheKeepBinaryIterationStoreEnabledTest.class);


[40/50] ignite git commit: Compatibility 7.5.26 fix

Posted by vo...@apache.org.
Compatibility 7.5.26 fix


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

Branch: refs/heads/ignite-3341
Commit: f7da0ddbeac3c47f322e794231fe209c2e5d7216
Parents: ee41656
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Jun 16 18:28:37 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Jun 16 18:28:37 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/lang/GridFunc.java     | 68 +++++++++++++++++++-
 1 file changed, 67 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f7da0ddb/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index 20e31c6..a3eae7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1695,6 +1695,72 @@ public class GridFunc {
     }
 
     /**
+     * Creates read-only light-weight view on given list with provided transformation.
+     * Resulting list will only "have" {@code transformed} elements. Note that only wrapping
+     * list will be created and no duplication of data will occur.
+     *
+     * @param c Input list that serves as a base for the view.
+     * @param trans Transformation closure.
+     * @param <T1> Type of the list.
+     * @return Light-weight view on given list with provided transformation.
+     */
+    @SuppressWarnings("RedundantTypeArguments")
+    @Deprecated
+    public static <T1, T2> List<T2> viewListReadOnly(@Nullable final List<? extends T1> c,
+        final IgniteClosure<? super T1, T2> trans) {
+        A.notNull(trans, "trans");
+
+        if (isEmpty(c))
+            return Collections.emptyList();
+
+        assert c != null;
+
+        return new GridSerializableList<T2>() {
+            /** */
+            private static final long serialVersionUID = 3126625219739967068L;
+
+            @Override public T2 get(int idx) {
+                return trans.apply(c.get(idx));
+            }
+
+            @NotNull
+            @Override public Iterator<T2> iterator() {
+                return F.<T1, T2>iterator(c, trans, true);
+            }
+
+            @Override public int size() {
+                return c.size();
+            }
+
+            @Override public boolean isEmpty() {
+                return c.isEmpty();
+            }
+        };
+    }
+
+    /**
+     * Creates a view on given list with provided transformer and predicates.
+     * Resulting list will only "have" elements for which all provided predicates, if any,
+     * evaluate to {@code true}. Note that a new collection will be created and data will
+     * be copied.
+     *
+     * @param c Input list that serves as a base for the view.
+     * @param trans Transforming closure from T1 to T2.
+     * @param p Optional predicates. If predicates are not provided - all elements will be in the view.
+     * @return View on given list with provided predicate.
+     */
+    @Deprecated
+    public static <T1, T2> List<T2> transformList(Collection<? extends T1> c,
+        IgniteClosure<? super T1, T2> trans, @Nullable IgnitePredicate<? super T1>... p) {
+        A.notNull(c, "c", trans, "trans");
+
+        if (isAlwaysFalse(p))
+            return Collections.emptyList();
+
+        return new ArrayList<>(transform(retain(c, true, p), trans));
+    }
+
+    /**
      * Creates light-weight view on given map with provided predicates. Resulting map will
      * only "have" keys for which all provided predicates, if any, evaluates to {@code true}.
      * Note that only wrapping map will be created and no duplication of data will occur.
@@ -4514,4 +4580,4 @@ public class GridFunc {
     public static IgnitePredicate<IgniteInternalFuture<?>> unfinishedFutures() {
         return UNFINISHED_FUTURE;
     }
-}
\ No newline at end of file
+}


[49/50] ignite git commit: FIxed.

Posted by vo...@apache.org.
FIxed.


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

Branch: refs/heads/ignite-3341
Commit: 699ff6a16d6b0edc46001d06cd9e0e45d7970ac3
Parents: c952def
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 20 12:22:38 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 20 12:22:38 2016 +0300

----------------------------------------------------------------------
 .../configuration/HadoopConfiguration.java      | 38 ++++++++++++++++++++
 .../processors/hadoop/HadoopJobInfo.java        |  5 ++-
 .../processors/hadoop/HadoopClassLoader.java    | 14 ++++++--
 .../processors/hadoop/HadoopDefaultJobInfo.java |  9 ++---
 .../hadoop/jobtracker/HadoopJobTracker.java     |  8 +++--
 .../processors/hadoop/v2/HadoopV2Job.java       | 14 ++++++--
 .../hadoop/HadoopClassLoaderTest.java           |  2 +-
 .../processors/hadoop/HadoopSnappyTest.java     |  2 +-
 .../collections/HadoopAbstractMapTest.java      |  7 ++--
 9 files changed, 83 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
index 51aa291..7b87ecb 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
@@ -17,8 +17,13 @@
 
 package org.apache.ignite.configuration;
 
+import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collection;
+import java.util.List;
 
 /**
  * Ignite Hadoop Accelerator configuration.
@@ -51,6 +56,9 @@ public class HadoopConfiguration {
     /** */
     private int maxTaskQueueSize = DFLT_MAX_TASK_QUEUE_SIZE;
 
+    /** Library names. */
+    private Collection<String> libNames;
+
     /**
      * Default constructor.
      */
@@ -71,6 +79,7 @@ public class HadoopConfiguration {
         planner = cfg.getMapReducePlanner();
         maxParallelTasks = cfg.getMaxParallelTasks();
         maxTaskQueueSize = cfg.getMaxTaskQueueSize();
+        libNames = cfg.getNativeLibraryNames();
     }
 
     /**
@@ -169,6 +178,35 @@ public class HadoopConfiguration {
         this.planner = planner;
     }
 
+    /**
+     * Get native library names.
+     * <p>
+     * Ignite Hadoop Accelerator executes all Hadoop jobs and tasks in the same process, isolating them with help
+     * of classloaders. If Hadoop job or task loads a native library, it might lead to exception, because Java do
+     * not allow to load the same library multiple times from different classloaders. To overcome the problem,
+     * you should to the following:
+     * <ul>
+     *     <li>Load necessary libraries in advance from base classloader; {@link LifecycleBean} is a good candidate
+     *     for this;</li>
+     *     <li>Add names of loaded libraries to this property, so that we can link them.</li>
+     *     <li>Remove {@link System#load(String)} and {@link System#loadLibrary(String)} calls from your job/task.</li>     *
+     * </ul>
+     *
+     * @return Native library names.
+     */
+    @Nullable public Collection<String> getNativeLibraryNames() {
+        return libNames;
+    }
+
+    /**
+     * Set native library names. See {@link #getNativeLibraryNames()} for more information.
+     *
+     * @param libNames Native library names.
+     */
+    public void setNativeLibraryNames(@Nullable Collection<String> libNames) {
+        this.libNames = libNames;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(HadoopConfiguration.class, this, super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
index eda8e97..832cb20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.processors.hadoop;
 
 import java.io.Serializable;
+import java.util.Collection;
+
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.jetbrains.annotations.Nullable;
@@ -58,11 +60,12 @@ public interface HadoopJobInfo extends Serializable {
      * @param jobCls The job class.
      * @param jobId Job ID.
      * @param log Logger.
+     * @param libNames Optional additional native library names.
      * @return Job.
      * @throws IgniteCheckedException If failed.
      */
     public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
-        HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
+        HadoopJobId jobId, IgniteLogger log, @Nullable Collection<String> libNames) throws IgniteCheckedException;
 
     /**
      * @return Number of reducers configured for job.

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index 4448b2d..fdbf06f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -99,6 +99,9 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
     @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"})
     private final String name;
 
+    /** Native library names. */
+    private final Collection<String> libNames;
+
     /**
      * Gets name for Job class loader. The name is specific for local node id.
      * @param locNodeId The local node id.
@@ -122,14 +125,19 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
     }
 
     /**
+     * Constructor.
+     *
      * @param urls Urls.
+     * @param name Classloader name.
+     * @param libNames Optional additional native library names to be linked from parent classloader.
      */
-    public HadoopClassLoader(URL[] urls, String name) {
+    public HadoopClassLoader(URL[] urls, String name, @Nullable Collection<String> libNames) {
         super(addHadoopUrls(urls), APP_CLS_LDR);
 
         assert !(getParent() instanceof HadoopClassLoader);
 
         this.name = name;
+        this.libNames = libNames;
 
         initializeNativeLibraries();
     }
@@ -161,7 +169,9 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
                 for (Object lib : vector) {
                     String libName = U.field(lib, "name");
 
-                    if (libName.contains(LIBHADOOP)) {
+                    boolean add = libName.contains(LIBHADOOP) || (libNames != null && libNames.contains(libName));
+
+                    if (add) {
                         curVector.add(lib);
 
                         return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
index fe125fe..6ca6f1a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.lang.reflect.Constructor;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
@@ -82,15 +83,15 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
-            HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+    @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls, HadoopJobId jobId, IgniteLogger log,
+        @Nullable Collection<String> libNames) throws IgniteCheckedException {
         assert jobCls != null;
 
         try {
             Constructor<? extends HadoopJob> constructor = jobCls.getConstructor(HadoopJobId.class,
-                HadoopDefaultJobInfo.class, IgniteLogger.class);
+                HadoopDefaultJobInfo.class, IgniteLogger.class, Collection.class);
 
-            return constructor.newInstance(jobId, this, log);
+            return constructor.newInstance(jobId, this, log, libNames);
         }
         // NB: java.lang.NoClassDefFoundError may be thrown from Class#getConstructor() call.
         catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index f4cf892..f3294bf 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -157,7 +157,8 @@ public class HadoopJobTracker extends HadoopComponent {
 
         assert jobCls == null;
 
-        HadoopClassLoader ldr = new HadoopClassLoader(null, HadoopClassLoader.nameForJob(nodeId));
+        HadoopClassLoader ldr = new HadoopClassLoader(null, HadoopClassLoader.nameForJob(nodeId),
+            ctx.configuration().getNativeLibraryNames());
 
         try {
             jobCls = (Class<HadoopV2Job>)ldr.loadClass(HadoopV2Job.class.getName());
@@ -727,6 +728,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param jobId  Job ID.
      * @param plan Map-reduce plan.
      */
+    @SuppressWarnings({"unused", "ConstantConditions" })
     private void printPlan(HadoopJobId jobId, HadoopMapReducePlan plan) {
         log.info("Plan for " + jobId);
 
@@ -886,6 +888,8 @@ public class HadoopJobTracker extends HadoopComponent {
                     finishFut.onDone(jobId, meta.failCause());
                 }
 
+                assert job != null;
+
                 if (ctx.jobUpdateLeader())
                     job.cleanupStagingDirectory();
 
@@ -1052,7 +1056,7 @@ public class HadoopJobTracker extends HadoopComponent {
                 jobInfo = meta.jobInfo();
             }
 
-            job = jobInfo.createJob(jobCls, jobId, log);
+            job = jobInfo.createJob(jobCls, jobId, log, ctx.configuration().getNativeLibraryNames());
 
             job.initialize(false, ctx.localNodeId());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
index b69447d..edd5269 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
@@ -93,6 +93,9 @@ public class HadoopV2Job implements HadoopJob {
     /** Job info. */
     protected final HadoopJobInfo jobInfo;
 
+    /** Native library names. */
+    private final Collection<String> libNames;
+
     /** */
     private final JobID hadoopJobID;
 
@@ -119,16 +122,21 @@ public class HadoopV2Job implements HadoopJob {
     private volatile byte[] jobConfData;
 
     /**
+     * Constructor.
+     *
      * @param jobId Job ID.
      * @param jobInfo Job info.
      * @param log Logger.
+     * @param libNames Optional additional native library names.
      */
-    public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log) {
+    public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log,
+        @Nullable Collection<String> libNames) {
         assert jobId != null;
         assert jobInfo != null;
 
         this.jobId = jobId;
         this.jobInfo = jobInfo;
+        this.libNames = libNames;
 
         hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId());
 
@@ -220,7 +228,7 @@ public class HadoopV2Job implements HadoopJob {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "MismatchedQueryAndUpdateOfCollection" })
     @Override public HadoopTaskContext getTaskContext(HadoopTaskInfo info) throws IgniteCheckedException {
         T2<HadoopTaskType, Integer> locTaskId = new T2<>(info.type(),  info.taskNumber());
 
@@ -242,7 +250,7 @@ public class HadoopV2Job implements HadoopJob {
                 // Note that the classloader identified by the task it was initially created for,
                 // but later it may be reused for other tasks.
                 HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath(),
-                    HadoopClassLoader.nameForTask(info, false));
+                    HadoopClassLoader.nameForTask(info, false), libNames);
 
                 cls = (Class<? extends HadoopTaskContext>)ldr.loadClass(HadoopV2TaskContext.class.getName());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
index 55fac2c..02d98d0 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
@@ -49,7 +49,7 @@ import org.apache.ignite.internal.processors.hadoop.deps.Without;
  */
 public class HadoopClassLoaderTest extends TestCase {
     /** */
-    final HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+    final HadoopClassLoader ldr = new HadoopClassLoader(null, "test", null);
 
     /**
      * @throws Exception If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java
index 014ff1e..b4e3dc2 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSnappyTest.java
@@ -49,7 +49,7 @@ public class HadoopSnappyTest extends GridCommonAbstractTest {
 
         // Run the same in several more class loaders simulating jobs and tasks:
         for (int i = 0; i < 2; i++) {
-            ClassLoader hadoopClsLdr = new HadoopClassLoader(null, "cl-" + i);
+            ClassLoader hadoopClsLdr = new HadoopClassLoader(null, "cl-" + i, null);
 
             Class<?> cls = (Class)Class.forName(HadoopSnappyTest.class.getName(), true, hadoopClsLdr);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/699ff6a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
index 493098f..a9d2116 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop.shuffle.collections;
 
+import java.util.Collection;
 import java.util.Comparator;
 import java.util.concurrent.Callable;
 import org.apache.commons.collections.comparators.ComparableComparator;
@@ -77,11 +78,13 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
         @Override public Comparator<Object> sortComparator() {
             return ComparableComparator.getInstance();
         }
 
         /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
         @Override public Comparator<Object> groupComparator() {
             return ComparableComparator.getInstance();
         }
@@ -141,8 +144,8 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
-                HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+        @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls, HadoopJobId jobId, IgniteLogger log,
+            @Nullable Collection<String> libNames) throws IgniteCheckedException {
             assert false;
 
             return null;


[30/50] ignite git commit: Merge branch 'gridgain-7.5.26' into ignite-3209

Posted by vo...@apache.org.
Merge branch 'gridgain-7.5.26' into ignite-3209


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

Branch: refs/heads/ignite-3341
Commit: 2c18b6ec363baf01300d5ce571f86f03afdeea97
Parents: 2a3ffc1 5d2bb23
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 15 16:44:49 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jun 15 16:44:49 2016 +0300

----------------------------------------------------------------------
 modules/cassandra/README.txt                    |  32 +
 modules/cassandra/licenses/apache-2.0.txt       | 202 +++++
 modules/cassandra/pom.xml                       | 340 +++++++
 .../store/cassandra/CassandraCacheStore.java    | 409 +++++++++
 .../cassandra/CassandraCacheStoreFactory.java   | 200 +++++
 .../store/cassandra/common/CassandraHelper.java | 133 +++
 .../cassandra/common/PropertyMappingHelper.java | 220 +++++
 .../store/cassandra/common/RandomSleeper.java   | 104 +++
 .../store/cassandra/common/SystemHelper.java    |  46 +
 .../store/cassandra/datasource/Credentials.java |  37 +
 .../store/cassandra/datasource/DataSource.java  | 550 ++++++++++++
 .../cassandra/datasource/PlainCredentials.java  |  50 ++
 .../persistence/KeyPersistenceSettings.java     | 274 ++++++
 .../KeyValuePersistenceSettings.java            | 478 ++++++++++
 .../persistence/PersistenceController.java      | 421 +++++++++
 .../persistence/PersistenceSettings.java        | 335 +++++++
 .../persistence/PersistenceStrategy.java        |  62 ++
 .../store/cassandra/persistence/PojoField.java  | 219 +++++
 .../cassandra/persistence/PojoKeyField.java     |  91 ++
 .../cassandra/persistence/PojoValueField.java   | 152 ++++
 .../persistence/ValuePersistenceSettings.java   | 107 +++
 .../cassandra/serializer/JavaSerializer.java    |  81 ++
 .../cassandra/serializer/KryoSerializer.java    |  93 ++
 .../store/cassandra/serializer/Serializer.java  |  43 +
 .../session/BatchExecutionAssistant.java        |  95 ++
 .../cassandra/session/BatchLoaderAssistant.java |  47 +
 .../cassandra/session/CassandraSession.java     |  60 ++
 .../cassandra/session/CassandraSessionImpl.java | 832 +++++++++++++++++
 .../cassandra/session/ExecutionAssistant.java   |  77 ++
 .../session/GenericBatchExecutionAssistant.java |  67 ++
 .../session/LoadCacheCustomQueryWorker.java     | 105 +++
 .../cassandra/session/pool/SessionPool.java     | 173 ++++
 .../cassandra/session/pool/SessionWrapper.java  |  71 ++
 .../store/cassandra/utils/DDLGenerator.java     |  64 ++
 .../aws/cassandra/cassandra-bootstrap.sh        | 373 ++++++++
 .../bootstrap/aws/cassandra/cassandra-env.sh    | 283 ++++++
 .../bootstrap/aws/cassandra/cassandra-start.sh  | 550 ++++++++++++
 .../aws/cassandra/cassandra-template.yaml       | 889 +++++++++++++++++++
 .../bootstrap/aws/ignite/ignite-bootstrap.sh    | 384 ++++++++
 .../ignite/ignite-cassandra-server-template.xml | 177 ++++
 .../src/test/bootstrap/aws/ignite/ignite-env.sh |  25 +
 .../test/bootstrap/aws/ignite/ignite-start.sh   | 637 +++++++++++++
 .../src/test/bootstrap/aws/logs-collector.sh    | 102 +++
 .../tests/ignite-cassandra-client-template.xml  | 173 ++++
 .../test/bootstrap/aws/tests/tests-bootstrap.sh | 379 ++++++++
 .../test/bootstrap/aws/tests/tests-report.sh    | 590 ++++++++++++
 .../src/test/bootstrap/aws/tests/tests-run.sh   | 715 +++++++++++++++
 .../CassandraDirectPersistenceLoadTest.java     | 107 +++
 .../tests/CassandraDirectPersistenceTest.java   | 371 ++++++++
 .../apache/ignite/tests/DDLGeneratorTest.java   |  43 +
 .../tests/IgnitePersistentStoreLoadTest.java    | 111 +++
 .../ignite/tests/IgnitePersistentStoreTest.java | 369 ++++++++
 .../org/apache/ignite/tests/load/Generator.java |  27 +
 .../apache/ignite/tests/load/IntGenerator.java  |  33 +
 .../ignite/tests/load/LoadTestDriver.java       | 238 +++++
 .../apache/ignite/tests/load/LongGenerator.java |  28 +
 .../ignite/tests/load/PersonGenerator.java      |  43 +
 .../ignite/tests/load/PersonIdGenerator.java    |  31 +
 .../ignite/tests/load/StringGenerator.java      |  28 +
 .../org/apache/ignite/tests/load/Worker.java    | 429 +++++++++
 .../tests/load/cassandra/BulkReadWorker.java    |  63 ++
 .../tests/load/cassandra/BulkWriteWorker.java   |  52 ++
 .../ignite/tests/load/cassandra/ReadWorker.java |  51 ++
 .../tests/load/cassandra/WriteWorker.java       |  51 ++
 .../tests/load/ignite/BulkReadWorker.java       |  52 ++
 .../tests/load/ignite/BulkWriteWorker.java      |  52 ++
 .../ignite/tests/load/ignite/ReadWorker.java    |  51 ++
 .../ignite/tests/load/ignite/WriteWorker.java   |  51 ++
 .../org/apache/ignite/tests/pojos/Person.java   | 246 +++++
 .../org/apache/ignite/tests/pojos/PersonId.java | 110 +++
 .../ignite/tests/utils/CacheStoreHelper.java    |  64 ++
 .../tests/utils/CassandraAdminCredentials.java  |  36 +
 .../ignite/tests/utils/CassandraHelper.java     | 358 ++++++++
 .../tests/utils/CassandraLifeCycleBean.java     | 149 ++++
 .../utils/CassandraRegularCredentials.java      |  36 +
 .../ignite/tests/utils/TestCacheSession.java    |  91 ++
 .../apache/ignite/tests/utils/TestsHelper.java  | 375 ++++++++
 .../src/test/resources/log4j.properties         | 119 +++
 .../tests/cassandra/connection-settings.xml     |  48 +
 .../tests/cassandra/connection.properties       |  17 +
 .../tests/cassandra/credentials.properties      |  22 +
 .../tests/cassandra/embedded-cassandra.yaml     | 119 +++
 .../ignite/tests/cassandra/keyspaces.properties |  17 +
 .../tests/persistence/blob/ignite-config.xml    | 100 +++
 .../persistence/blob/persistence-settings-1.xml |  21 +
 .../persistence/blob/persistence-settings-2.xml |  21 +
 .../persistence/blob/persistence-settings-3.xml |  29 +
 .../tests/persistence/pojo/ignite-config.xml    | 119 +++
 .../persistence/pojo/persistence-settings-1.xml |  21 +
 .../persistence/pojo/persistence-settings-2.xml |  21 +
 .../persistence/pojo/persistence-settings-3.xml | 173 ++++
 .../persistence/primitive/ignite-config.xml     | 100 +++
 .../primitive/ignite-remote-client-config.xml   |  95 ++
 .../primitive/ignite-remote-server-config.xml   | 106 +++
 .../primitive/persistence-settings-1.xml        |  21 +
 .../primitive/persistence-settings-2.xml        |  21 +
 .../src/test/resources/tests.properties         |  50 ++
 .../src/test/scripts/cassandra-load-tests.bat   |  41 +
 .../src/test/scripts/cassandra-load-tests.sh    |  39 +
 .../src/test/scripts/ignite-load-tests.bat      |  41 +
 .../src/test/scripts/ignite-load-tests.sh       |  39 +
 modules/cassandra/src/test/scripts/jvm-opt.sh   |  21 +
 modules/cassandra/src/test/scripts/jvm-opts.bat |  24 +
 .../processors/igfs/IgfsMetaManager.java        |  40 +-
 .../processors/query/h2/sql/GridSqlConst.java   |   5 +
 .../processors/query/h2/sql/GridSqlJoin.java    |  17 +-
 .../processors/query/h2/sql/GridSqlType.java    |   5 +
 .../query/IgniteSqlSplitterSelfTest.java        |  75 ++
 .../org/apache/ignite/spark/IgniteContext.scala |   4 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |  25 +-
 .../apache/ignite/spark/JavaIgniteContext.scala |   4 +-
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |   2 +
 .../ignite/spark/impl/IgniteAbstractRDD.scala   |  15 +-
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |   5 +-
 .../spark/impl/JavaIgniteAbstractRDD.scala      |  34 -
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |  25 +-
 pom.xml                                         |   9 +
 117 files changed, 16935 insertions(+), 68 deletions(-)
----------------------------------------------------------------------



[08/50] ignite git commit: Tcp discovery minor: create buffered input stream with size = socket receiveBufferSize.

Posted by vo...@apache.org.
Tcp discovery minor: create buffered input stream with size = socket receiveBufferSize.


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

Branch: refs/heads/ignite-3341
Commit: 54425bf3e4b92c99feb3be1fd697ec5e93f9e456
Parents: 6625e01
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 14 11:42:52 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 14 11:43:20 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/54425bf3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index b7814e5..2ce2e7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -5180,7 +5180,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                     for (IgniteInClosure<Socket> connLsnr : spi.incomeConnLsnrs)
                         connLsnr.apply(sock);
 
-                    in = new BufferedInputStream(sock.getInputStream());
+                    int rcvBufSize = sock.getReceiveBufferSize();
+
+                    in = new BufferedInputStream(sock.getInputStream(), rcvBufSize > 0 ? rcvBufSize : 8192);
 
                     byte[] buf = new byte[4];
                     int read = 0;


[19/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
new file mode 100644
index 0000000..d1ef1bd
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java
@@ -0,0 +1,358 @@
+/*
+ * 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.tests.utils;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import java.lang.reflect.Field;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.ResourceBundle;
+import org.apache.ignite.cache.store.cassandra.bean.CassandraLifeCycleBean;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.session.pool.SessionPool;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lifecycle.LifecycleEventType;
+import org.apache.ignite.logger.log4j.Log4JLogger;
+import org.apache.log4j.Logger;
+import org.springframework.context.ApplicationContext;
+import org.springframework.context.support.ClassPathXmlApplicationContext;
+
+/**
+ * Helper class providing bunch of utility methods to work with Cassandra
+ */
+public class CassandraHelper {
+    /** */
+    private static final ResourceBundle CREDENTIALS = ResourceBundle.getBundle("org/apache/ignite/tests/cassandra/credentials");
+
+    /** */
+    private static final ResourceBundle CONNECTION = ResourceBundle.getBundle("org/apache/ignite/tests/cassandra/connection");
+
+    /** */
+    private static final ResourceBundle KEYSPACES = ResourceBundle.getBundle("org/apache/ignite/tests/cassandra/keyspaces");
+
+    /** */
+    private static final String EMBEDDED_CASSANDRA_YAML = "org/apache/ignite/tests/cassandra/embedded-cassandra.yaml";
+
+    /** */
+    private static final ApplicationContext connectionContext = new ClassPathXmlApplicationContext("org/apache/ignite/tests/cassandra/connection-settings.xml");
+
+    /** */
+    private static DataSource adminDataSrc;
+
+    /** */
+    private static DataSource regularDataSrc;
+
+    /** */
+    private static Cluster adminCluster;
+
+    /** */
+    private static Cluster regularCluster;
+
+    /** */
+    private static Session adminSes;
+
+    /** */
+    private static Session regularSes;
+
+    /** */
+    private static CassandraLifeCycleBean embeddedCassandraBean;
+
+    /** */
+    public static String getAdminUser() {
+        return CREDENTIALS.getString("admin.user");
+    }
+
+    /** */
+    public static String getAdminPassword() {
+        return CREDENTIALS.getString("admin.password");
+    }
+
+    /** */
+    public static String getRegularUser() {
+        return CREDENTIALS.getString("regular.user");
+    }
+
+    /** */
+    public static String getRegularPassword() {
+        return CREDENTIALS.getString("regular.password");
+    }
+
+    /** */
+    public static String[] getTestKeyspaces() {
+        return KEYSPACES.getString("keyspaces").split(",");
+    }
+
+    /** */
+    public static String[] getContactPointsArray() {
+        String[] points = CONNECTION.getString("contact.points").split(",");
+
+        if (points.length == 0)
+            throw new RuntimeException("No Cassandra contact points specified");
+
+        for (int i = 0; i < points.length; i++)
+            points[i] = points[i].trim();
+
+        return points;
+    }
+
+    /** */
+    public static List<InetAddress> getContactPoints() {
+        String[] points = getContactPointsArray();
+
+        List<InetAddress> contactPoints = new LinkedList<>();
+
+        for (String point : points) {
+            if (point.contains(":"))
+                continue;
+
+            try {
+                contactPoints.add(InetAddress.getByName(point));
+            }
+            catch (Throwable e) {
+                throw new IllegalArgumentException("Incorrect contact point '" + point +
+                    "' specified for Cassandra cache storage", e);
+            }
+        }
+
+        return contactPoints;
+    }
+
+    /** */
+    public static List<InetSocketAddress> getContactPointsWithPorts() {
+        String[] points = getContactPointsArray();
+
+        List<InetSocketAddress> contactPoints = new LinkedList<>();
+
+        for (String point : points) {
+            if (!point.contains(":"))
+                continue;
+
+            String[] chunks = point.split(":");
+
+            try {
+                contactPoints.add(InetSocketAddress.createUnresolved(chunks[0].trim(), Integer.parseInt(chunks[1].trim())));
+            }
+            catch (Throwable e) {
+                throw new IllegalArgumentException("Incorrect contact point '" + point +
+                    "' specified for Cassandra cache storage", e);
+            }
+        }
+
+        return contactPoints;
+    }
+
+    /**
+     * Checks if embedded Cassandra should be used for unit tests
+     * @return true if embedded Cassandra should be used
+     */
+    public static boolean useEmbeddedCassandra() {
+        String[] contactPoints = getContactPointsArray();
+
+        return contactPoints != null && contactPoints.length == 1 && contactPoints[0].trim().startsWith("127.0.0.1");
+    }
+
+    /** */
+    public static void dropTestKeyspaces() {
+        String[] keyspaces = getTestKeyspaces();
+
+        for (String keyspace : keyspaces) {
+            try {
+                executeWithAdminCredentials("DROP KEYSPACE IF EXISTS " + keyspace + ";");
+            }
+            catch (Throwable e) {
+                throw new RuntimeException("Failed to drop keyspace: " + keyspace, e);
+            }
+        }
+    }
+
+    /** */
+    public static ResultSet executeWithAdminCredentials(String statement, Object... args) {
+        if (args == null || args.length == 0)
+            return adminSession().execute(statement);
+
+        PreparedStatement ps = adminSession().prepare(statement);
+        return adminSession().execute(ps.bind(args));
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public static ResultSet executeWithRegularCredentials(String statement, Object... args) {
+        if (args == null || args.length == 0)
+            return regularSession().execute(statement);
+
+        PreparedStatement ps = regularSession().prepare(statement);
+        return regularSession().execute(ps.bind(args));
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public static ResultSet executeWithAdminCredentials(Statement statement) {
+        return adminSession().execute(statement);
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public static ResultSet executeWithRegularCredentials(Statement statement) {
+        return regularSession().execute(statement);
+    }
+
+    /** */
+    public static synchronized DataSource getAdminDataSrc() {
+        if (adminDataSrc != null)
+            return adminDataSrc;
+
+        return adminDataSrc = (DataSource)connectionContext.getBean("cassandraAdminDataSource");
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public static synchronized DataSource getRegularDataSrc() {
+        if (regularDataSrc != null)
+            return regularDataSrc;
+
+        return regularDataSrc = (DataSource)connectionContext.getBean("cassandraRegularDataSource");
+    }
+
+    /** */
+    public static void testAdminConnection() {
+        try {
+            adminSession();
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to check admin connection to Cassandra", e);
+        }
+    }
+
+    /** */
+    public static void testRegularConnection() {
+        try {
+            regularSession();
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to check regular connection to Cassandra", e);
+        }
+    }
+
+    /** */
+    public static synchronized void releaseCassandraResources() {
+        try {
+            if (adminSes != null && !adminSes.isClosed())
+                U.closeQuiet(adminSes);
+        }
+        finally {
+            adminSes = null;
+        }
+
+        try {
+            if (adminCluster != null && !adminCluster.isClosed())
+                U.closeQuiet(adminCluster);
+        }
+        finally {
+            adminCluster = null;
+        }
+
+        try {
+            if (regularSes != null && !regularSes.isClosed())
+                U.closeQuiet(regularSes);
+        }
+        finally {
+            regularSes = null;
+        }
+
+        try {
+            if (regularCluster != null && !regularCluster.isClosed())
+                U.closeQuiet(regularCluster);
+        }
+        finally {
+            regularCluster = null;
+        }
+
+        SessionPool.release();
+    }
+
+    /** */
+    private static synchronized Session adminSession() {
+        if (adminSes != null)
+            return adminSes;
+
+        try {
+            Cluster.Builder builder = Cluster.builder();
+            builder = builder.withCredentials(getAdminUser(), getAdminPassword());
+            builder.addContactPoints(getContactPoints());
+            builder.addContactPointsWithPorts(getContactPointsWithPorts());
+
+            adminCluster = builder.build();
+            return adminSes = adminCluster.connect();
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to create admin session to Cassandra database", e);
+        }
+    }
+
+    /** */
+    private static synchronized Session regularSession() {
+        if (regularSes != null)
+            return regularSes;
+
+        try {
+            Cluster.Builder builder = Cluster.builder();
+            builder = builder.withCredentials(getRegularUser(), getRegularPassword());
+            builder.addContactPoints(getContactPoints());
+            builder.addContactPointsWithPorts(getContactPointsWithPorts());
+
+            regularCluster = builder.build();
+            return regularSes = regularCluster.connect();
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to create regular session to Cassandra database", e);
+        }
+    }
+
+    /** */
+    public static void startEmbeddedCassandra(Logger log) {
+        ClassLoader clsLdr = CassandraHelper.class.getClassLoader();
+        URL url = clsLdr.getResource(EMBEDDED_CASSANDRA_YAML);
+
+        embeddedCassandraBean = new CassandraLifeCycleBean();
+        embeddedCassandraBean.setCassandraConfigFile(url.getFile());
+
+        try {
+            Field logField = CassandraLifeCycleBean.class.getDeclaredField("log");
+            logField.setAccessible(true);
+            logField.set(embeddedCassandraBean, new Log4JLogger(log));
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to initialize logger for CassandraLifeCycleBean", e);
+        }
+
+        embeddedCassandraBean.onLifecycleEvent(LifecycleEventType.BEFORE_NODE_START);
+    }
+
+    /** */
+    public static void stopEmbeddedCassandra() {
+        if (embeddedCassandraBean != null)
+            embeddedCassandraBean.onLifecycleEvent(LifecycleEventType.BEFORE_NODE_STOP);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java
new file mode 100644
index 0000000..52937ea
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tests.utils;
+
+import org.apache.ignite.cache.store.cassandra.datasource.Credentials;
+
+/**
+ * Implementation of {@link org.apache.ignite.cache.store.cassandra.datasource.Credentials}
+ * providing regular user/password to establish Cassandra session.
+ */
+public class CassandraRegularCredentials implements Credentials {
+    /** {@inheritDoc} */
+    @Override public String getUser() {
+        return CassandraHelper.getRegularUser();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getPassword() {
+        return CassandraHelper.getRegularPassword();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
new file mode 100644
index 0000000..1cedb7a
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
@@ -0,0 +1,91 @@
+/*
+ * 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.tests.utils;
+
+import java.util.Map;
+import org.apache.ignite.cache.store.CacheStoreSession;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of {@link CacheStoreSession} for the unit tests purposes
+ */
+public class TestCacheSession implements CacheStoreSession {
+    /** */
+    private String cacheName;
+
+    /** */
+    private Transaction tx;
+
+    /** */
+    private Map<Object, Object> props;
+
+    /** */
+    private Object attach;
+
+    /** */
+    public TestCacheSession(String cacheName) {
+        this.cacheName = cacheName;
+    }
+
+    /** */
+    @SuppressWarnings("UnusedDeclaration")
+    public void newSession(@Nullable Transaction tx) {
+        this.tx = tx;
+        props = null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Transaction transaction() {
+        return tx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWithinTransaction() {
+        return transaction() != null;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Object attach(@Nullable Object attach) {
+        Object prev = this.attach;
+        this.attach = attach;
+        return prev;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T attachment() {
+        return (T) attach;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <K, V> Map<K, V> properties() {
+        if (props == null)
+            props = U.newHashMap(1);
+
+        return (Map<K, V>)props;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String cacheName() {
+        return cacheName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
new file mode 100644
index 0000000..0bbda7f
--- /dev/null
+++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
@@ -0,0 +1,375 @@
+/*
+ * 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.tests.utils;
+
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.ResourceBundle;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.tests.load.Generator;
+import org.apache.ignite.tests.pojos.Person;
+import org.apache.ignite.tests.pojos.PersonId;
+import org.springframework.core.io.ClassPathResource;
+
+/**
+ * Helper class for all tests
+ */
+public class TestsHelper {
+    /** */
+    private static final String LETTERS_ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ";
+
+    /** */
+    private static final String NUMBERS_ALPHABET = "0123456789";
+
+    /** */
+    private static final Random RANDOM = new Random(System.currentTimeMillis());
+
+    /** */
+    private static final ResourceBundle TESTS_SETTINGS = ResourceBundle.getBundle("tests");
+
+    /** */
+    private static final int BULK_OPERATION_SIZE = parseTestSettings("bulk.operation.size");
+
+    /** */
+    private static final String LOAD_TESTS_CACHE_NAME = TESTS_SETTINGS.getString("load.tests.cache.name");
+
+    /** */
+    private static final int LOAD_TESTS_THREADS_COUNT = parseTestSettings("load.tests.threads.count");
+
+    /** */
+    private static final int LOAD_TESTS_WARMUP_PERIOD = parseTestSettings("load.tests.warmup.period");
+
+    /** */
+    private static final int LOAD_TESTS_EXECUTION_TIME = parseTestSettings("load.tests.execution.time");
+
+    /** */
+    private static final int LOAD_TESTS_REQUESTS_LATENCY = parseTestSettings("load.tests.requests.latency");
+
+    /** */
+    private static final String LOAD_TESTS_PERSISTENCE_SETTINGS = TESTS_SETTINGS.getString("load.tests.persistence.settings");
+
+    /** */
+    private static final String LOAD_TESTS_IGNITE_CONFIG = TESTS_SETTINGS.getString("load.tests.ignite.config");
+
+    /** */
+    private static final Generator LOAD_TESTS_KEY_GENERATOR;
+
+    /** */
+    private static final Generator LOAD_TESTS_VALUE_GENERATOR;
+
+    /** */
+    private static int parseTestSettings(String name) {
+        return Integer.parseInt(TESTS_SETTINGS.getString(name));
+    }
+
+    static {
+        try {
+            LOAD_TESTS_KEY_GENERATOR = (Generator)Class.forName(TESTS_SETTINGS.getString("load.tests.key.generator")).newInstance();
+            LOAD_TESTS_VALUE_GENERATOR = (Generator)Class.forName(TESTS_SETTINGS.getString("load.tests.value.generator")).newInstance();
+        }
+        catch (Throwable e) {
+            throw new RuntimeException("Failed to initialize TestsHelper", e);
+        }
+    }
+
+    /** */
+    public static int getLoadTestsThreadsCount() {
+        return LOAD_TESTS_THREADS_COUNT;
+    }
+
+    /** */
+    public static int getLoadTestsWarmupPeriod() {
+        return LOAD_TESTS_WARMUP_PERIOD;
+    }
+
+    /** */
+    public static int getLoadTestsExecutionTime() {
+        return LOAD_TESTS_EXECUTION_TIME;
+    }
+
+    /** */
+    public static int getLoadTestsRequestsLatency() {
+        return LOAD_TESTS_REQUESTS_LATENCY;
+    }
+
+    /** */
+    public static ClassPathResource getLoadTestsPersistenceSettings() {
+        return new ClassPathResource(LOAD_TESTS_PERSISTENCE_SETTINGS);
+    }
+
+    /** */
+    public static String getLoadTestsIgniteConfig() {
+        return LOAD_TESTS_IGNITE_CONFIG;
+    }
+
+    /** */
+    public static int getBulkOperationSize() {
+        return BULK_OPERATION_SIZE;
+    }
+
+    /** */
+    public static String getLoadTestsCacheName() {
+        return LOAD_TESTS_CACHE_NAME;
+    }
+
+    /** */
+    public static Object generateLoadTestsKey(long i) {
+        return LOAD_TESTS_KEY_GENERATOR.generate(i);
+    }
+
+    /** */
+    public static Object generateLoadTestsValue(long i) {
+        return LOAD_TESTS_VALUE_GENERATOR.generate(i);
+    }
+
+    /** */
+    @SuppressWarnings("unchecked")
+    public static CacheEntryImpl generateLoadTestsEntry(long i) {
+        return new CacheEntryImpl(TestsHelper.generateLoadTestsKey(i), TestsHelper.generateLoadTestsValue(i));
+    }
+
+    /** */
+    public static <K, V> Collection<K> getKeys(Collection<CacheEntryImpl<K, V>> entries) {
+        List<K> list = new LinkedList<>();
+
+        for (CacheEntryImpl<K, ?> entry : entries)
+            list.add(entry.getKey());
+
+        return list;
+    }
+
+    /** */
+    public static Map<Long, Long> generateLongsMap() {
+        return generateLongsMap(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Map<Long, Long> generateLongsMap(int cnt) {
+        Map<Long, Long> map = new HashMap<>();
+
+        for (long i = 0; i < cnt; i++)
+            map.put(i, i + 123);
+
+        return map;
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<Long, Long>> generateLongsEntries() {
+        return generateLongsEntries(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<Long, Long>> generateLongsEntries(int cnt) {
+        Collection<CacheEntryImpl<Long, Long>> entries = new LinkedList<>();
+
+        for (long i = 0; i < cnt; i++)
+            entries.add(new CacheEntryImpl<>(i, i + 123));
+
+        return entries;
+    }
+
+    /** */
+    public static Map<String, String> generateStringsMap() {
+        return generateStringsMap(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Map<String, String> generateStringsMap(int cnt) {
+        Map<String, String> map = new HashMap<>();
+
+        for (int i = 0; i < cnt; i++)
+            map.put(Integer.toString(i), randomString(5));
+
+        return map;
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<String, String>> generateStringsEntries() {
+        return generateStringsEntries(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<String, String>> generateStringsEntries(int cnt) {
+        Collection<CacheEntryImpl<String, String>> entries = new LinkedList<>();
+
+        for (int i = 0; i < cnt; i++)
+            entries.add(new CacheEntryImpl<>(Integer.toString(i), randomString(5)));
+
+        return entries;
+    }
+
+    /** */
+    public static Map<Long, Person> generateLongsPersonsMap() {
+        Map<Long, Person> map = new HashMap<>();
+
+        for (long i = 0; i < BULK_OPERATION_SIZE; i++)
+            map.put(i, generateRandomPerson());
+
+        return map;
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<Long, Person>> generateLongsPersonsEntries() {
+        Collection<CacheEntryImpl<Long, Person>> entries = new LinkedList<>();
+
+        for (long i = 0; i < BULK_OPERATION_SIZE; i++)
+            entries.add(new CacheEntryImpl<>(i, generateRandomPerson()));
+
+        return entries;
+    }
+
+    /** */
+    public static Map<PersonId, Person> generatePersonIdsPersonsMap() {
+        return generatePersonIdsPersonsMap(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Map<PersonId, Person> generatePersonIdsPersonsMap(int cnt) {
+        Map<PersonId, Person> map = new HashMap<>();
+
+        for (int i = 0; i < cnt; i++)
+            map.put(generateRandomPersonId(), generateRandomPerson());
+
+        return map;
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<PersonId, Person>> generatePersonIdsPersonsEntries() {
+        return generatePersonIdsPersonsEntries(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<PersonId, Person>> generatePersonIdsPersonsEntries(int cnt) {
+        Collection<CacheEntryImpl<PersonId, Person>> entries = new LinkedList<>();
+
+        for (int i = 0; i < cnt; i++)
+            entries.add(new CacheEntryImpl<>(generateRandomPersonId(), generateRandomPerson()));
+
+        return entries;
+    }
+
+    /** */
+    public static Person generateRandomPerson() {
+        int phonesCnt = RANDOM.nextInt(4);
+
+        List<String> phones = new LinkedList<>();
+
+        for (int i = 0; i < phonesCnt; i++)
+            phones.add(randomNumber(4));
+
+        return new Person(randomString(4), randomString(4), RANDOM.nextInt(100),
+            RANDOM.nextBoolean(), RANDOM.nextLong(), RANDOM.nextFloat(), new Date(), phones);
+    }
+
+    /** */
+    public static PersonId generateRandomPersonId() {
+        return new PersonId(randomString(4), randomString(4), RANDOM.nextInt(100));
+    }
+
+    /** */
+    public static boolean checkMapsEqual(Map map1, Map map2) {
+        if (map1 == null || map2 == null || map1.size() != map2.size())
+            return false;
+
+        for (Object key : map1.keySet()) {
+            Object obj1 = map1.get(key);
+            Object obj2 = map2.get(key);
+
+            if (obj1 == null || obj2 == null || !obj1.equals(obj2))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    public static <K, V> boolean checkCollectionsEqual(Map<K, V> map, Collection<CacheEntryImpl<K, V>> col) {
+        if (map == null || col == null || map.size() != col.size())
+            return false;
+
+        for (CacheEntryImpl<K, V> entry : col) {
+            if (!entry.getValue().equals(map.get(entry.getKey())))
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    public static <K> boolean checkPersonMapsEqual(Map<K, Person> map1, Map<K, Person> map2,
+        boolean primitiveFieldsOnly) {
+        if (map1 == null || map2 == null || map1.size() != map2.size())
+            return false;
+
+        for (K key : map1.keySet()) {
+            Person person1 = map1.get(key);
+            Person person2 = map2.get(key);
+
+            boolean equals = person1 != null && person2 != null &&
+                (primitiveFieldsOnly ? person1.equalsPrimitiveFields(person2) : person1.equals(person2));
+
+            if (!equals)
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    public static <K> boolean checkPersonCollectionsEqual(Map<K, Person> map, Collection<CacheEntryImpl<K, Person>> col,
+        boolean primitiveFieldsOnly) {
+        if (map == null || col == null || map.size() != col.size())
+            return false;
+
+        for (CacheEntryImpl<K, Person> entry : col) {
+            boolean equals = primitiveFieldsOnly ?
+                entry.getValue().equalsPrimitiveFields(map.get(entry.getKey())) :
+                entry.getValue().equals(map.get(entry.getKey()));
+
+            if (!equals)
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    public static String randomString(int len) {
+        StringBuilder builder = new StringBuilder(len);
+
+        for (int i = 0; i < len; i++)
+            builder.append(LETTERS_ALPHABET.charAt(RANDOM.nextInt(LETTERS_ALPHABET.length())));
+
+        return builder.toString();
+    }
+
+    /** */
+    public static String randomNumber(int len) {
+        StringBuilder builder = new StringBuilder(len);
+
+        for (int i = 0; i < len; i++)
+            builder.append(NUMBERS_ALPHABET.charAt(RANDOM.nextInt(NUMBERS_ALPHABET.length())));
+
+        return builder.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/log4j.properties b/modules/cassandra/src/test/resources/log4j.properties
new file mode 100644
index 0000000..9c67be4
--- /dev/null
+++ b/modules/cassandra/src/test/resources/log4j.properties
@@ -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.
+
+# Root logger option
+log4j.rootLogger=INFO, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+# ------ LOAD TESTS LOGGING ------
+
+# Cassandra load tests loggers
+log4j.logger.CassandraLoadTests=INFO, stdout, cassandraLoadTests
+log4j.logger.CassandraWriteLoadTest=INFO, csWrite
+log4j.logger.CassandraBulkWriteLoadTest=INFO, csBulkWrite
+log4j.logger.CassandraReadLoadTest=INFO, csRead
+log4j.logger.CassandraBulkReadLoadTest=INFO, csBulkRead
+
+log4j.appender.cassandraLoadTests=org.apache.log4j.RollingFileAppender
+log4j.appender.cassandraLoadTests.File=logs/cassandra-load-tests.log
+log4j.appender.cassandraLoadTests.MaxFileSize=10MB
+log4j.appender.cassandraLoadTests.MaxBackupIndex=10
+log4j.appender.cassandraLoadTests.Append=true
+log4j.appender.cassandraLoadTests.layout=org.apache.log4j.PatternLayout
+log4j.appender.cassandraLoadTests.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.csWrite=org.apache.log4j.RollingFileAppender
+log4j.appender.csWrite.File=logs/cassandra-write.log
+log4j.appender.csWrite.MaxFileSize=10MB
+log4j.appender.csWrite.MaxBackupIndex=10
+log4j.appender.csWrite.Append=true
+log4j.appender.csWrite.layout=org.apache.log4j.PatternLayout
+log4j.appender.csWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.csBulkWrite=org.apache.log4j.RollingFileAppender
+log4j.appender.csBulkWrite.File=logs/cassandra-bulk-write.log
+log4j.appender.csBulkWrite.MaxFileSize=10MB
+log4j.appender.csBulkWrite.MaxBackupIndex=10
+log4j.appender.csBulkWrite.Append=true
+log4j.appender.csBulkWrite.layout=org.apache.log4j.PatternLayout
+log4j.appender.csBulkWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.csRead=org.apache.log4j.RollingFileAppender
+log4j.appender.csRead.File=logs/cassandra-read.log
+log4j.appender.csRead.MaxFileSize=10MB
+log4j.appender.csRead.MaxBackupIndex=10
+log4j.appender.csRead.Append=true
+log4j.appender.csRead.layout=org.apache.log4j.PatternLayout
+log4j.appender.csRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.csBulkRead=org.apache.log4j.RollingFileAppender
+log4j.appender.csBulkRead.File=logs/cassandra-bulk-read.log
+log4j.appender.csBulkRead.MaxFileSize=10MB
+log4j.appender.csBulkRead.MaxBackupIndex=10
+log4j.appender.csBulkRead.Append=true
+log4j.appender.csBulkRead.layout=org.apache.log4j.PatternLayout
+log4j.appender.csBulkRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+# Ignite load tests loggers
+log4j.logger.IgniteLoadTests=INFO, stdout, igniteLoadTests
+log4j.logger.IgniteWriteLoadTest=INFO, igWrite
+log4j.logger.IgniteBulkWriteLoadTest=INFO, igBulkWrite
+log4j.logger.IgniteReadLoadTest=INFO, igRead
+log4j.logger.IgniteBulkReadLoadTest=INFO, igBulkRead
+
+log4j.appender.igniteLoadTests=org.apache.log4j.RollingFileAppender
+log4j.appender.igniteLoadTests.File=logs/ignite-load-tests.log
+log4j.appender.igniteLoadTests.MaxFileSize=10MB
+log4j.appender.igniteLoadTests.MaxBackupIndex=10
+log4j.appender.igniteLoadTests.Append=true
+log4j.appender.igniteLoadTests.layout=org.apache.log4j.PatternLayout
+log4j.appender.igniteLoadTests.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.igWrite=org.apache.log4j.RollingFileAppender
+log4j.appender.igWrite.File=logs/ignite-write.log
+log4j.appender.igWrite.MaxFileSize=10MB
+log4j.appender.igWrite.MaxBackupIndex=10
+log4j.appender.igWrite.Append=true
+log4j.appender.igWrite.layout=org.apache.log4j.PatternLayout
+log4j.appender.igWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.igBulkWrite=org.apache.log4j.RollingFileAppender
+log4j.appender.igBulkWrite.File=logs/ignite-bulk-write.log
+log4j.appender.igBulkWrite.MaxFileSize=10MB
+log4j.appender.igBulkWrite.MaxBackupIndex=10
+log4j.appender.igBulkWrite.Append=true
+log4j.appender.igBulkWrite.layout=org.apache.log4j.PatternLayout
+log4j.appender.igBulkWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.igRead=org.apache.log4j.RollingFileAppender
+log4j.appender.igRead.File=logs/ignite-read.log
+log4j.appender.igRead.MaxFileSize=10MB
+log4j.appender.igRead.MaxBackupIndex=10
+log4j.appender.igRead.Append=true
+log4j.appender.igRead.layout=org.apache.log4j.PatternLayout
+log4j.appender.igRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n
+
+log4j.appender.igBulkRead=org.apache.log4j.RollingFileAppender
+log4j.appender.igBulkRead.File=logs/ignite-bulk-read.log
+log4j.appender.igBulkRead.MaxFileSize=10MB
+log4j.appender.igBulkRead.MaxBackupIndex=10
+log4j.appender.igBulkRead.Append=true
+log4j.appender.igBulkRead.layout=org.apache.log4j.PatternLayout
+log4j.appender.igBulkRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml
new file mode 100644
index 0000000..f7eb372
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <bean id="cassandraAdminCredentials" class="org.apache.ignite.tests.utils.CassandraAdminCredentials"/>
+    <bean id="cassandraRegularCredentials" class="org.apache.ignite.tests.utils.CassandraRegularCredentials"/>
+
+    <bean id="loadBalancingPolicy" class="com.datastax.driver.core.policies.RoundRobinPolicy"/>
+
+    <bean id="contactPoints" class="org.apache.ignite.tests.utils.CassandraHelper" factory-method="getContactPointsArray"/>
+
+    <bean id="cassandraAdminDataSource" class="org.apache.ignite.cache.store.cassandra.datasource.DataSource">
+        <property name="credentials" ref="cassandraAdminCredentials"/>
+        <property name="contactPoints" ref="contactPoints"/>
+        <property name="readConsistency" value="ONE"/>
+        <property name="writeConsistency" value="ONE"/>
+        <property name="loadBalancingPolicy" ref="loadBalancingPolicy"/>
+    </bean>
+
+    <bean id="cassandraRegularDataSource" class="org.apache.ignite.cache.store.cassandra.datasource.DataSource">
+        <property name="credentials" ref="cassandraRegularCredentials"/>
+        <property name="contactPoints" ref="contactPoints"/>
+        <property name="readConsistency" value="ONE"/>
+        <property name="writeConsistency" value="ONE"/>
+        <property name="loadBalancingPolicy" ref="loadBalancingPolicy"/>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties
new file mode 100644
index 0000000..ef15018
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties
@@ -0,0 +1,17 @@
+# 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.
+
+# Comma delimited Cassandra contact points in format: host[:port]
+contact.points=127.0.0.1

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties
new file mode 100644
index 0000000..f011bcc
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties
@@ -0,0 +1,22 @@
+# 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.
+
+# Cassandra admin user/password
+admin.user=
+admin.password=
+
+# Cassandra regular user/password
+regular.user=
+regular.password=

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/embedded-cassandra.yaml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/embedded-cassandra.yaml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/embedded-cassandra.yaml
new file mode 100644
index 0000000..293098a
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/embedded-cassandra.yaml
@@ -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.
+
+cluster_name: 'Test Cluster'
+
+listen_address: 127.0.0.1
+native_transport_port: 9042
+rpc_address: 127.0.0.1
+rpc_port: 9160
+
+seed_provider:
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          - seeds: "127.0.0.1"
+saved_caches_directory: ./data/saved_caches
+commitlog_directory: ./data/commitlog
+hints_directory: ./data/hints
+data_file_directories:
+    - ./data/data
+
+num_tokens: 256
+hinted_handoff_enabled: true
+max_hint_window_in_ms: 10800000 # 3 hours
+hinted_handoff_throttle_in_kb: 1024
+max_hints_delivery_threads: 2
+hints_flush_period_in_ms: 10000
+max_hints_file_size_in_mb: 128
+batchlog_replay_throttle_in_kb: 1024
+authenticator: AllowAllAuthenticator
+authorizer: AllowAllAuthorizer
+role_manager: CassandraRoleManager
+roles_validity_in_ms: 2000
+permissions_validity_in_ms: 2000
+partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+disk_failure_policy: stop
+commit_failure_policy: stop
+key_cache_size_in_mb:
+key_cache_save_period: 14400
+row_cache_size_in_mb: 0
+row_cache_save_period: 0
+counter_cache_size_in_mb:
+counter_cache_save_period: 7200
+commitlog_sync: periodic
+commitlog_sync_period_in_ms: 10000
+commitlog_segment_size_in_mb: 32
+concurrent_reads: 32
+concurrent_writes: 32
+concurrent_counter_writes: 32
+concurrent_materialized_view_writes: 32
+memtable_allocation_type: heap_buffers
+index_summary_capacity_in_mb:
+index_summary_resize_interval_in_minutes: 60
+trickle_fsync: false
+trickle_fsync_interval_in_kb: 10240
+storage_port: 7000
+ssl_storage_port: 7001
+start_native_transport: true
+start_rpc: false
+rpc_keepalive: true
+rpc_server_type: sync
+thrift_framed_transport_size_in_mb: 15
+incremental_backups: false
+snapshot_before_compaction: false
+auto_snapshot: true
+tombstone_warn_threshold: 1000
+tombstone_failure_threshold: 100000
+column_index_size_in_kb: 64
+batch_size_warn_threshold_in_kb: 5
+batch_size_fail_threshold_in_kb: 50
+compaction_throughput_mb_per_sec: 16
+compaction_large_partition_warning_threshold_mb: 100
+sstable_preemptive_open_interval_in_mb: 50
+read_request_timeout_in_ms: 5000
+range_request_timeout_in_ms: 10000
+write_request_timeout_in_ms: 2000
+counter_write_request_timeout_in_ms: 5000
+cas_contention_timeout_in_ms: 1000
+truncate_request_timeout_in_ms: 60000
+request_timeout_in_ms: 10000
+cross_node_timeout: false
+endpoint_snitch: SimpleSnitch
+dynamic_snitch_update_interval_in_ms: 100
+dynamic_snitch_reset_interval_in_ms: 600000
+dynamic_snitch_badness_threshold: 0.1
+request_scheduler: org.apache.cassandra.scheduler.NoScheduler
+
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+
+client_encryption_options:
+    enabled: false
+    optional: false
+    keystore: conf/.keystore
+    keystore_password: cassandra
+
+internode_compression: all
+inter_dc_tcp_nodelay: false
+tracetype_query_ttl: 86400
+tracetype_repair_ttl: 604800
+gc_warn_threshold_in_ms: 1000
+enable_user_defined_functions: false
+enable_scripted_user_defined_functions: false
+windows_timer_interval: 1

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties
new file mode 100644
index 0000000..9205cc1
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties
@@ -0,0 +1,17 @@
+# 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.
+
+# Cassandra keyspaces used for tests
+keyspaces=test1

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml
new file mode 100644
index 0000000..9aa5c84
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <!-- Cassandra connection settings -->
+    <import resource="classpath:org/apache/ignite/tests/cassandra/connection-settings.xml" />
+
+    <!-- Persistence settings for 'cache1' -->
+    <bean id="cache1_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml" />
+    </bean>
+
+    <!-- Persistence settings for 'cache2' -->
+    <bean id="cache2_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml" />
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Partitioned cache example configuration (Atomic mode). -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache2" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache2_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery. For information on all options refer
+                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml
new file mode 100644
index 0000000..1c1951d
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml
@@ -0,0 +1,21 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="blob_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" />
+    <valuePersistence strategy="BLOB"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml
new file mode 100644
index 0000000..49b3caf
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml
@@ -0,0 +1,21 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="blob_test2">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE"/>
+    <valuePersistence strategy="BLOB"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml
new file mode 100644
index 0000000..905c3e5
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml
@@ -0,0 +1,29 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="blob_test3">
+    <!-- By default Java standard serialization is going to be used -->
+    <keyPersistence class="java.lang.Long"
+                    strategy="BLOB"
+                    column="key"/>
+
+    <!-- Kryo serialization specified to be used -->
+    <valuePersistence class="org.apache.ignite.tests.pojos.Person"
+                      strategy="BLOB"
+                      serializer="org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer"
+                      column="value"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
new file mode 100644
index 0000000..8dcfffd
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
@@ -0,0 +1,119 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <!-- Cassandra connection settings -->
+    <import resource="classpath:org/apache/ignite/tests/cassandra/connection-settings.xml" />
+
+    <!-- Persistence settings for 'cache1' -->
+    <bean id="cache1_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml" />
+    </bean>
+
+    <!-- Persistence settings for 'cache2' -->
+    <bean id="cache2_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml" />
+    </bean>
+
+    <!-- Persistence settings for 'cache3' -->
+    <bean id="cache3_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml" />
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Partitioned cache example configuration (Atomic mode). -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache2" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache2_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache3" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache3"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache3_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery. For information on all options refer
+                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml
new file mode 100644
index 0000000..b39578c
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml
@@ -0,0 +1,21 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="pojo_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE"/>
+    <valuePersistence class="org.apache.ignite.tests.pojos.Person" strategy="POJO"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml
new file mode 100644
index 0000000..10a2d9f
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml
@@ -0,0 +1,21 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="pojo_test2">
+    <keyPersistence class="org.apache.ignite.tests.pojos.PersonId" strategy="POJO"/>
+    <valuePersistence class="org.apache.ignite.tests.pojos.Person" strategy="POJO"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml
new file mode 100644
index 0000000..dd8eac5
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml
@@ -0,0 +1,173 @@
+<!--
+  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.
+-->
+
+<!--
+Root container for persistence settings configuration.
+
+Note: required element
+
+Attributes:
+  1) keyspace [required] - keyspace for Cassandra tables which should be used to store key/value pairs
+  2) table    [required] - Cassandra tables which should be used to store key/value pairs
+  3) ttl      [optional] - expiration period for the table rows (in seconds)
+-->
+<persistence keyspace="test1" table="pojo_test3" ttl="86400">
+    <!--
+    Cassandra keyspace options which should be used to create provided keyspace if it doesn't exist.
+
+    Note: optional element
+    -->
+    <keyspaceOptions>
+        REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 3}
+        AND DURABLE_WRITES = true
+    </keyspaceOptions>
+
+    <!--
+    Cassandra table options which should be used to create provided table if it doesn't exist.
+
+    Note: optional element
+    -->
+    <tableOptions>
+        comment = 'A most excellent and useful table'
+        AND read_repair_chance = 0.2
+    </tableOptions>
+
+    <!--
+    Persistent settings for Ignite cache keys.
+
+    Note: required element
+
+    Attributes:
+      1) class      [required] - java class name for Ignite cache key
+      2) strategy   [required] - one of three possible persistent strategies which controls how object
+        should be persisted/loaded to/from Cassandra table:
+            a) PRIMITIVE - stores key value as is, by mapping it to Cassandra table column with corresponding type.
+                Should be used only for simple java types (int, long, String, double, Date) which could be mapped
+                to corresponding Cassadra types.
+            b) BLOB - stores key value as BLOB, by mapping it to Cassandra table column with blob type.
+                Could be used for any java object. Conversion of java object to BLOB is handled by "serializer"
+                which could be specified in serializer attribute (see below).
+            c) POJO - stores each field of an object as a column having corresponding type in Cassandra table.
+                Provides ability to utilize Cassandra secondary indexes for object fields.
+      3) serializer [optional] - specifies serializer class for BLOB strategy. Shouldn't be used for PRIMITIVE and
+        POJO strategies. Available implementations:
+            a) org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer - uses standard Java
+                serialization framework
+            b) org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer - uses Kryo
+                serialization framework
+      4) column     [optional] - specifies column name for PRIMITIVE and BLOB strategies where to store key value.
+        If not specified column having 'key' name will be used. Shouldn't be used for POJO strategy.
+    -->
+    <keyPersistence class="org.apache.ignite.tests.pojos.PersonId" strategy="POJO">
+        <!--
+        Partition key fields if POJO strategy used.
+
+        Note: optional element, only required for POJO strategy in case you want to manually specify
+            POJO fields to Cassandra columns mapping, instead of relying on dynamic discovering of
+            POJO fields and mapping them to the same columns of Cassandra table.
+        -->
+        <partitionKey>
+            <!--
+             Mapping from POJO field to Cassandra table column.
+
+             Note: required element
+
+             Attributes:
+               1) name   [required] - POJO field name
+               2) column [optional] - Cassandra table column name. If not specified lowercase
+                  POJO field name will be used.
+            -->
+            <field name="companyCode" column="company" />
+            <field name="departmentCode" column="department" />
+        </partitionKey>
+
+        <!--
+        Cluster key fields if POJO strategy used.
+
+        Note: optional element, only required for POJO strategy in case you want to manually specify
+            POJO fields to Cassandra columns mapping, instead of relying on dynamic discovering of
+            POJO fields and mapping them to the same columns of Cassandra table.
+        -->
+        <clusterKey>
+            <!--
+             Mapping from POJO field to Cassandra table column.
+
+             Note: required element
+
+             Attributes:
+               1) name   [required] - POJO field name
+               2) column [optional] - Cassandra table column name. If not specified lowercase
+                  POJO field name will be used.
+               3) sort   [optional] - specifies sort order (**asc** or **desc**)
+            -->
+            <field name="personNumber" column="number" sort="desc"/>
+        </clusterKey>
+    </keyPersistence>
+
+    <!--
+    Persistent settings for Ignite cache values.
+
+    Note: required element
+
+    Attributes:
+      1) class      [required] - java class name for Ignite cache value
+      2) strategy   [required] - one of three possible persistent strategies which controls how object
+        should be persisted/loaded to/from Cassandra table:
+            a) PRIMITIVE - stores key value as is, by mapping it to Cassandra table column with corresponding type.
+                Should be used only for simple java types (int, long, String, double, Date) which could be mapped
+                to corresponding Cassadra types.
+            b) BLOB - stores key value as BLOB, by mapping it to Cassandra table column with blob type.
+                Could be used for any java object. Conversion of java object to BLOB is handled by "serializer"
+                which could be specified in serializer attribute (see below).
+            c) POJO - stores each field of an object as a column having corresponding type in Cassandra table.
+                Provides ability to utilize Cassandra secondary indexes for object fields.
+      3) serializer [optional] - specifies serializer class for BLOB strategy. Shouldn't be used for PRIMITIVE and
+        POJO strategies. Available implementations:
+            a) org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer - uses standard Java
+                serialization framework
+            b) org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer - uses Kryo
+                serialization framework
+      4) column     [optional] - specifies column name for PRIMITIVE and BLOB strategies where to store value.
+        If not specified column having 'value' name will be used. Shouldn't be used for POJO strategy.
+    -->
+    <valuePersistence class="org.apache.ignite.tests.pojos.Person"
+                      strategy="POJO"
+                      serializer="org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer">
+        <!--
+         Mapping from POJO field to Cassandra table column.
+
+         Note: required element
+
+         Attributes:
+           1) name         [required] - POJO field name
+           2) column       [optional] - Cassandra table column name. If not specified lowercase
+              POJO field name will be used.
+           3) static       [optional] - boolean flag which specifies that column is static withing a given partition
+           4) index        [optional] - boolean flag specifying that secondary index should be created for the field
+           5) indexClass   [optional] - custom index java class name, in case you want to use custom index
+           6) indexOptions [optional] - custom index options
+        -->
+        <field name="firstName" column="first_name" />
+        <field name="lastName" column="last_name" />
+        <field name="age" />
+        <field name="married" index="true"/>
+        <field name="height" />
+        <field name="weight" />
+        <field name="birthDate" column="birth_date" />
+        <field name="phones" />
+    </valuePersistence>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml
new file mode 100644
index 0000000..fb6b055
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml
@@ -0,0 +1,100 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <!-- Cassandra connection settings -->
+    <import resource="classpath:org/apache/ignite/tests/cassandra/connection-settings.xml" />
+
+    <!-- Persistence settings for 'cache1' -->
+    <bean id="cache1_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml" />
+    </bean>
+
+    <!-- Persistence settings for 'cache2' -->
+    <bean id="cache2_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml" />
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Partitioned cache example configuration (Atomic mode). -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache2" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache2_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery. For information on all options refer
+                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>


[50/50] ignite git commit: Finalization.

Posted by vo...@apache.org.
Finalization.


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

Branch: refs/heads/ignite-3341
Commit: d4d6b0c54204a413acd2e3ca093dc29a9935333c
Parents: 699ff6a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jun 20 15:50:52 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jun 20 15:50:52 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/hadoop/HadoopClassLoader.java | 14 ++++++++++++--
 .../external/child/HadoopChildProcessRunner.java      |  2 +-
 .../processors/hadoop/HadoopMapReduceTest.java        |  3 +++
 .../internal/processors/hadoop/HadoopTasksV1Test.java |  2 +-
 .../internal/processors/hadoop/HadoopTasksV2Test.java |  2 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java        |  2 +-
 6 files changed, 19 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d6b0c5/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index fdbf06f..bf9eb46 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -167,9 +167,19 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
                 Vector vector = U.field(ldr, "nativeLibraries");
 
                 for (Object lib : vector) {
-                    String libName = U.field(lib, "name");
+                    String name = U.field(lib, "name");
 
-                    boolean add = libName.contains(LIBHADOOP) || (libNames != null && libNames.contains(libName));
+                    boolean add = name.contains(LIBHADOOP);
+
+                    if (!add && libNames != null) {
+                        for (String libName : libNames) {
+                            if (libName != null && name.contains(libName)) {
+                                add = true;
+
+                                break;
+                            }
+                        }
+                    }
 
                     if (add) {
                         curVector.add(lib);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d6b0c5/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index 550c3ba..b1f68c7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -134,7 +134,7 @@ public class HadoopChildProcessRunner {
 
                 assert job == null;
 
-                job = req.jobInfo().createJob(HadoopV2Job.class, req.jobId(), log);
+                job = req.jobInfo().createJob(HadoopV2Job.class, req.jobId(), log, null);
 
                 job.initialize(true, nodeDesc.processId());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d6b0c5/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
index 4426847..1cb4f74 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
@@ -20,7 +20,9 @@ package org.apache.ignite.internal.processors.hadoop;
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
@@ -37,6 +39,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.configuration.HadoopConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter;
 import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d6b0c5/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
index 6ba9686..27d7fc2 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
@@ -48,7 +48,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
 
         HadoopJobId jobId = new HadoopJobId(uuid, 0);
 
-        return jobInfo.createJob(HadoopV2Job.class, jobId, log);
+        return jobInfo.createJob(HadoopV2Job.class, jobId, log, null);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d6b0c5/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
index d125deb..30cf50c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
@@ -67,7 +67,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
 
         HadoopJobId jobId = new HadoopJobId(uuid, 0);
 
-        return jobInfo.createJob(HadoopV2Job.class, jobId, log);
+        return jobInfo.createJob(HadoopV2Job.class, jobId, log, null);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d4d6b0c5/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
index 1e9ffbc..ae2c00d 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
@@ -78,7 +78,7 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
 
         HadoopJobId id = new HadoopJobId(uuid, 1);
 
-        HadoopJob job = info.createJob(HadoopV2Job.class, id, log);
+        HadoopJob job = info.createJob(HadoopV2Job.class, id, log, null);
 
         HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0,
             null));


[29/50] ignite git commit: IGNITE-3312: IGFS: Fixed a bug introduced during create() routine reworking.

Posted by vo...@apache.org.
IGNITE-3312: IGFS: Fixed a bug introduced during create() routine reworking.


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

Branch: refs/heads/ignite-3341
Commit: 5d2bb234218c6ff6a4e04613b166e4a7487d5081
Parents: 95d5750
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jun 15 16:02:53 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jun 15 16:02:53 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 40 +++++++++++++++-----
 1 file changed, 31 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5d2bb234/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 39ec182..943c4c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -67,6 +67,7 @@ import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T1;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -2754,7 +2755,7 @@ public class IgfsMetaManager extends IgfsManager {
                         else {
                             // Create file and parent folders.
                             IgfsPathsCreateResult res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
-                                affKey, evictExclude, null);
+                                affKey, evictExclude, null, null);
 
                             if (res == null)
                                 continue;
@@ -2918,11 +2919,21 @@ public class IgfsMetaManager extends IgfsManager {
                         }
                         else {
                             // Create file and parent folders.
+                            T1<OutputStream> secondaryOutHolder = null;
+
                             if (secondaryCtx != null)
-                                secondaryOut = secondaryCtx.create();
+                                secondaryOutHolder = new T1<>();
 
-                            IgfsPathsCreateResult res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
-                                affKey, evictExclude, secondaryCtx);
+                            IgfsPathsCreateResult res;
+
+                            try {
+                                res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
+                                    affKey, evictExclude, secondaryCtx, secondaryOutHolder);
+                            }
+                            finally {
+                                if (secondaryOutHolder != null)
+                                    secondaryOut =  secondaryOutHolder.get();
+                            }
 
                             if (res == null)
                                 continue;
@@ -2972,7 +2983,7 @@ public class IgfsMetaManager extends IgfsManager {
             throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
                 "element is not a directory)");
 
-        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false, null);
+        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false, null, null);
     }
 
     /**
@@ -2986,12 +2997,14 @@ public class IgfsMetaManager extends IgfsManager {
      * @param affKey Affinity key (optional)
      * @param evictExclude Evict exclude flag.
      * @param secondaryCtx Secondary file system create context.
+     * @param secondaryOutHolder Holder for the secondary output stream.
      * @return Result or {@code} if the first parent already contained child with the same name.
      * @throws IgniteCheckedException If failed.
      */
     @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map<IgniteUuid, IgfsEntryInfo> lockInfos,
         Map<String, String> dirProps, Map<String, String> fileProps, int blockSize, @Nullable IgniteUuid affKey,
-        boolean evictExclude, @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx)
+        boolean evictExclude, @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx,
+        @Nullable T1<OutputStream> secondaryOutHolder)
         throws IgniteCheckedException{
         // Check if entry we are going to write to is directory.
         if (lockInfos.get(pathIds.lastExistingId()).isFile())
@@ -2999,7 +3012,7 @@ public class IgfsMetaManager extends IgfsManager {
                 "(parent element is not a directory): " + pathIds.path());
 
         return createFileOrDirectory(false, pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude,
-            secondaryCtx);
+            secondaryCtx, secondaryOutHolder);
     }
 
     /**
@@ -3014,6 +3027,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @param affKey Affinity key.
      * @param evictExclude Evict exclude flag.
      * @param secondaryCtx Secondary file system create context.
+     * @param secondaryOutHolder Secondary output stream holder.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
@@ -3021,7 +3035,8 @@ public class IgfsMetaManager extends IgfsManager {
     private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pathIds,
         Map<IgniteUuid, IgfsEntryInfo> lockInfos, Map<String, String> dirProps, Map<String, String> fileProps,
         int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude,
-        @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx) throws IgniteCheckedException {
+        @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx, @Nullable T1<OutputStream> secondaryOutHolder)
+        throws IgniteCheckedException {
         // This is our starting point.
         int lastExistingIdx = pathIds.lastExistingIndex();
         IgfsEntryInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId());
@@ -3037,6 +3052,13 @@ public class IgfsMetaManager extends IgfsManager {
         if (lastExistingInfo.hasChild(curPart))
             return null;
 
+        // Create entry in the secondary file system if needed.
+        if (secondaryCtx != null) {
+            assert secondaryOutHolder != null;
+
+            secondaryOutHolder.set(secondaryCtx.create());
+        }
+
         Map<IgniteUuid, EntryProcessor> procMap = new HashMap<>();
 
         // First step: add new entry to the last existing element.
@@ -3068,7 +3090,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 if (secondaryInfo == null)
                     throw new IgfsException("Failed to perform operation because secondary file system path was " +
-                        "modified concurrnetly: " + lastCreatedPath);
+                        "modified concurrently: " + lastCreatedPath);
                 else if (secondaryInfo.isFile())
                     throw new IgfsException("Failed to perform operation because secondary file system entity is " +
                         "not directory: " + lastCreatedPath);


[43/50] ignite git commit: IGNITE-3334 - TcpDiscoveryZookeeperIpFinder used INFO logging without isInfoEnabled() check

Posted by vo...@apache.org.
IGNITE-3334 - TcpDiscoveryZookeeperIpFinder used INFO logging without isInfoEnabled() check


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

Branch: refs/heads/ignite-3341
Commit: 694aabb9060cad0a78eeaec005e7dd837dfc5472
Parents: df60f65
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Fri Jun 17 14:57:45 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Jun 17 14:57:45 2016 +0300

----------------------------------------------------------------------
 .../zk/TcpDiscoveryZookeeperIpFinder.java        | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/694aabb9/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
index 238987b..e7a732a 100644
--- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
+++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/TcpDiscoveryZookeeperIpFinder.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.spi.discovery.tcp.ipfinder.zk;
 
-import com.google.common.collect.Sets;
 import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Collections;
@@ -26,6 +25,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.collect.Sets;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
@@ -134,7 +134,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
         if (sysPropZkConnString != null && sysPropZkConnString.trim().length() > 0)
             zkConnectionString = sysPropZkConnString;
 
-        log.info("Initializing ZooKeeper IP Finder.");
+        if (log.isInfoEnabled())
+            log.info("Initializing ZooKeeper IP Finder.");
 
         if (curator == null) {
             A.notNullOrEmpty(zkConnectionString, String.format("ZooKeeper URL (or system property %s) cannot be null " +
@@ -162,7 +163,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
             return;
         }
 
-        log.info("Destroying ZooKeeper IP Finder.");
+        if (log.isInfoEnabled())
+            log.info("Destroying ZooKeeper IP Finder.");
 
         super.onSpiContextDestroyed();
 
@@ -193,7 +195,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
         for (ServiceInstance<IgniteInstanceDetails> si : serviceInstances)
             answer.add(new InetSocketAddress(si.getAddress(), si.getPort()));
 
-        log.info("ZooKeeper IP Finder resolved addresses: " + answer);
+        if (log.isInfoEnabled())
+            log.info("ZooKeeper IP Finder resolved addresses: " + answer);
 
         return answer;
     }
@@ -202,7 +205,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
     @Override public void registerAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException {
         init();
 
-        log.info("Registering addresses with ZooKeeper IP Finder: " + addrs);
+        if (log.isInfoEnabled())
+            log.info("Registering addresses with ZooKeeper IP Finder: " + addrs);
 
         Set<InetSocketAddress> registrationsToIgnore = Sets.newHashSet();
         if (!allowDuplicateRegistrations) {
@@ -248,7 +252,8 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
         if (curator.getState() != CuratorFrameworkState.STARTED)
             return;
 
-        log.info("Unregistering addresses with ZooKeeper IP Finder: " + addrs);
+        if (log.isInfoEnabled())
+            log.info("Unregistering addresses with ZooKeeper IP Finder: " + addrs);
 
         for (InetSocketAddress addr : addrs) {
             ServiceInstance<IgniteInstanceDetails> si = ourInstances.get(addr);
@@ -362,4 +367,4 @@ public class TcpDiscoveryZookeeperIpFinder extends TcpDiscoveryIpFinderAdapter {
 
     }
 
-}
\ No newline at end of file
+}


[33/50] ignite git commit: IGNITE-3151: Using IgniteCountDownLatch sometimes drives to dead lock. Reviewed and merged by Denis Magda.

Posted by vo...@apache.org.
IGNITE-3151: Using IgniteCountDownLatch sometimes drives to dead lock.
Reviewed and merged by Denis Magda.


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

Branch: refs/heads/ignite-3341
Commit: 314794bfc71a04a3c185abdbafe1f81b17cc0ec4
Parents: fb23e00
Author: Vladislav Pyatkov <vl...@gmail.com>
Authored: Wed Jun 15 18:43:22 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jun 15 18:43:22 2016 +0300

----------------------------------------------------------------------
 .../GridCacheCountDownLatchImpl.java            |  54 ++++++-
 .../IgniteCountDownLatchAbstractSelfTest.java   | 156 ++++++++++++++++++-
 2 files changed, 202 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/314794bf/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index c984ab3..5adeb38 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -26,7 +26,7 @@ import java.io.ObjectStreamException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
@@ -51,6 +51,15 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Internal latch is in unitialized state. */
+    private static final int UNINITIALIZED_LATCH_STATE = 0;
+
+    /** Internal latch is being created. */
+    private static final int CREATING_LATCH_STATE = 1;
+
+    /** Internal latch is ready for the usage. */
+    private static final int READY_LATCH_STATE = 2;
+
     /** Deserialization stash. */
     private static final ThreadLocal<IgniteBiTuple<GridKernalContext, String>> stash =
         new ThreadLocal<IgniteBiTuple<GridKernalContext, String>>() {
@@ -84,14 +93,17 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     private boolean autoDel;
 
     /** Internal latch (transient). */
-    private volatile CountDownLatch internalLatch;
+    private CountDownLatch internalLatch;
 
     /** Initialization guard. */
-    private final AtomicBoolean initGuard = new AtomicBoolean();
+    private final AtomicInteger initGuard = new AtomicInteger();
 
     /** Initialization latch. */
     private final CountDownLatch initLatch = new CountDownLatch(1);
 
+    /** Latest latch value that is used at the stage while the internal latch is being initialized. */
+    private Integer lastLatchVal = null;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -237,15 +249,36 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     @Override public void onUpdate(int cnt) {
         assert cnt >= 0;
 
-        while (internalLatch != null && internalLatch.getCount() > cnt)
-            internalLatch.countDown();
+        CountDownLatch latch0;
+
+        synchronized (initGuard) {
+            int state = initGuard.get();
+
+            if (state != READY_LATCH_STATE) {
+                /** Internal latch is not fully initialized yet. Remember latest latch value. */
+                lastLatchVal = cnt;
+
+                return;
+            }
+
+            /** 'synchronized' statement guarantees visibility of internalLatch. No need to make it volatile. */
+            latch0 = internalLatch;
+        }
+
+        /** Internal latch is fully initialized and ready for the usage. */
+
+        assert latch0 != null;
+
+        while (latch0.getCount() > cnt)
+            latch0.countDown();
+
     }
 
     /**
      * @throws IgniteCheckedException If operation failed.
      */
     private void initializeLatch() throws IgniteCheckedException {
-        if (initGuard.compareAndSet(false, true)) {
+        if (initGuard.compareAndSet(UNINITIALIZED_LATCH_STATE, CREATING_LATCH_STATE)) {
             try {
                 internalLatch = CU.outTx(
                     retryTopologySafe(new Callable<CountDownLatch>() {
@@ -269,6 +302,15 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
                     ctx
                 );
 
+                synchronized (initGuard) {
+                    if (lastLatchVal != null) {
+                        while (internalLatch.getCount() > lastLatchVal)
+                            internalLatch.countDown();
+                    }
+
+                    initGuard.set(READY_LATCH_STATE);
+                }
+
                 if (log.isDebugEnabled())
                     log.debug("Initialized internal latch: " + internalLatch);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/314794bf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
index 2f6f6f4..f6d0287 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
@@ -22,21 +22,26 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.Callable;
 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.IgniteCompute;
 import org.apache.ignite.IgniteCountDownLatch;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -144,7 +149,6 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
 
     /**
      * @param latch Latch.
-     *
      * @throws Exception If failed.
      */
     protected void checkRemovedLatch(final IgniteCountDownLatch latch) throws Exception {
@@ -236,8 +240,8 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
      * @param latchName Latch name.
      * @param cnt Count.
      * @param autoDel Auto delete flag.
-     * @throws Exception If failed.
      * @return New latch.
+     * @throws Exception If failed.
      */
     private IgniteCountDownLatch createLatch(String latchName, int cnt, boolean autoDel)
         throws Exception {
@@ -334,6 +338,58 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
     /**
      * @throws Exception If failed.
      */
+    public void testLatchBroadcast() throws Exception {
+        Ignite ignite = grid(0);
+        ClusterGroup srvsGrp = ignite.cluster().forServers();
+
+        int numOfSrvs = srvsGrp.nodes().size();
+
+        ignite.destroyCache("testCache");
+        IgniteCache<Object, Object> cache = ignite.createCache("testCache");
+
+        for (ClusterNode node : srvsGrp.nodes())
+            cache.put(String.valueOf(node.id()), 0);
+
+        for (int i = 0; i < 500; i++) {
+            IgniteCountDownLatch latch1 = createLatch1(ignite, numOfSrvs);
+            IgniteCountDownLatch latch2 = createLatch2(ignite, numOfSrvs);
+
+            ignite.compute(srvsGrp).broadcast(new IgniteRunnableJob(latch1, latch2, i));
+            assertTrue(latch2.await(10000));
+        }
+    }
+
+    /**
+     * @param client Ignite client.
+     * @param numOfSrvs Number of server nodes.
+     * @return Ignite latch.
+     */
+    private IgniteCountDownLatch createLatch1(Ignite client, int numOfSrvs) {
+        return client.countDownLatch(
+            "testName1", // Latch name.
+            numOfSrvs,          // Initial count.
+            true,        // Auto remove, when counter has reached zero.
+            true         // Create if it does not exist.
+        );
+    }
+
+    /**
+     * @param client Ignite client.
+     * @param numOfSrvs Number of server nodes.
+     * @return Ignite latch.
+     */
+    private IgniteCountDownLatch createLatch2(Ignite client, int numOfSrvs) {
+        return client.countDownLatch(
+            "testName2", // Latch name.
+            numOfSrvs,          // Initial count.
+            true,        // Auto remove, when counter has reached zero.
+            true         // Create if it does not exist.
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testLatchMultinode2() throws Exception {
         if (gridCount() == 1)
             return;
@@ -391,4 +447,100 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         // No-op.
     }
+
+    /**
+     * Ignite job
+     */
+    public class IgniteRunnableJob implements IgniteRunnable {
+
+        /**
+         * Ignite.
+         */
+        @IgniteInstanceResource
+        Ignite igniteInstance;
+
+        /**
+         * Number of iteration.
+         */
+        protected final int iteration;
+
+        /**
+         * Ignite latch 1.
+         */
+        private final IgniteCountDownLatch latch1;
+
+        /**
+         * Ignite latch 2.
+         */
+        private final IgniteCountDownLatch latch2;
+
+        /**
+         * @param latch1 Ignite latch 1.
+         * @param latch2 Ignite latch 2.
+         * @param iteration Number of iteration.
+         */
+        public IgniteRunnableJob(IgniteCountDownLatch latch1, IgniteCountDownLatch latch2, int iteration) {
+            this.iteration = iteration;
+            this.latch1 = latch1;
+            this.latch2 = latch2;
+        }
+
+        /**
+         * @return Ignite latch.
+         */
+        IgniteCountDownLatch createLatch1() {
+            return latch1;
+        }
+
+        /**
+         * @return Ignite latch.
+         */
+        IgniteCountDownLatch createLatch2() {
+            return latch2;
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        public void run() {
+
+            IgniteCountDownLatch latch1 = createLatch1();
+            IgniteCountDownLatch latch2 = createLatch2();
+
+            IgniteCache<Object, Object> cache = igniteInstance.cache("testCache");
+
+            for (ClusterNode node : igniteInstance.cluster().forServers().nodes()) {
+                Integer val = (Integer)cache.get(String.valueOf(node.id()));
+                assertEquals(val, (Integer)iteration);
+            }
+
+            latch1.countDown();
+
+            assertTrue(latch1.await(10000));
+
+            cache.put(getUID(), (iteration + 1));
+
+            latch2.countDown();
+
+        }
+
+        /**
+         * @return Node UUID as string.
+         */
+        String getUID() {
+            String id = "";
+            Collection<ClusterNode> nodes = igniteInstance.cluster().forLocal().nodes();
+            for (ClusterNode node : nodes) {
+                if (node.isLocal())
+                    id = String.valueOf(node.id());
+            }
+            return id;
+        }
+
+        /**
+         * @return Ignite.
+         */
+        public Ignite igniteInstance() {
+            return igniteInstance;
+        }
+    }
 }


[14/50] ignite git commit: ignite-3209 Review.

Posted by vo...@apache.org.
ignite-3209 Review.


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

Branch: refs/heads/ignite-3341
Commit: ebe5658ab22488ea58f46b3fa29705b44c04f451
Parents: e7b85c2
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 15 10:45:35 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 15 10:45:35 2016 +0300

----------------------------------------------------------------------
 .../processors/task/GridTaskWorker.java         |  4 +-
 .../cache/CacheAffinityCallSelfTest.java        | 41 +++++++++++++++-----
 2 files changed, 34 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebe5658a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 651259d..0c522ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -1064,9 +1064,9 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         IgniteInternalFuture<?> affFut = null;
 
         if (affKey != null) {
-            Long topVer = ctx.discovery().topologyVersion();
+            AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
 
-            affFut = ctx.cache().context().exchange().affinityReadyFuture(new AffinityTopologyVersion(topVer));
+            affFut = ctx.cache().context().exchange().affinityReadyFuture(topVer);
         }
 
         if (affFut == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebe5658a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
index e4b6ece..8621aa0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
@@ -68,18 +68,20 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
         AlwaysFailoverSpi failSpi = new AlwaysFailoverSpi();
         cfg.setFailoverSpi(failSpi);
 
-        CacheConfiguration ccfg = defaultCacheConfiguration();
-        ccfg.setName(CACHE_NAME);
-        ccfg.setCacheMode(PARTITIONED);
-        ccfg.setBackups(1);
-
-        cfg.setCacheConfiguration(ccfg);
-
+        // Do not configure cache on client.
         if (gridName.equals(getTestGridName(SERVERS_COUNT))) {
             cfg.setClientMode(true);
 
             spi.setForceServerMode(true);
         }
+        else {
+            CacheConfiguration ccfg = defaultCacheConfiguration();
+            ccfg.setName(CACHE_NAME);
+            ccfg.setCacheMode(PARTITIONED);
+            ccfg.setBackups(1);
+
+            cfg.setCacheConfiguration(ccfg);
+        }
 
         return cfg;
     }
@@ -95,7 +97,28 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
     public void testAffinityCallRestartNode() throws Exception {
         startGridsMultiThreaded(SERVERS_COUNT);
 
-        final int ITERS = 5;
+        affinityCallRestartNode(grid(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCallFromClientRestartNode() throws Exception {
+        startGridsMultiThreaded(SERVERS_COUNT + 1);
+
+        Ignite client = grid(SERVERS_COUNT);
+
+        assertTrue(client.configuration().isClientMode());
+
+        affinityCallRestartNode(client);
+    }
+
+    /**
+     * @param node Node executing affinity call.
+     * @throws Exception If failed.
+     */
+    private void affinityCallRestartNode(Ignite node) throws Exception {
+        final int ITERS = 10;
 
         for (int i = 0; i < ITERS; i++) {
             log.info("Iteration: " + i);
@@ -115,7 +138,7 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
             });
 
             while (!fut.isDone())
-                grid(1).compute().affinityCall(CACHE_NAME, key, new CheckCallable(key, topVer, topVer + 1));
+                node.compute().affinityCall(CACHE_NAME, key, new CheckCallable(key, topVer, topVer + 1));
 
             fut.get();
 


[44/50] ignite git commit: IGNITE-3335: IGFS: Improved affinity task execution for read-only operations from clients.

Posted by vo...@apache.org.
IGNITE-3335: IGFS: Improved affinity task execution for read-only operations from clients.


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

Branch: refs/heads/ignite-3341
Commit: 06d0e5112c2722dc94a754bb9b5ee952471ac2cd
Parents: df60f65
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 16:33:47 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 16:33:47 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDeleteWorker.java       |  2 +-
 .../processors/igfs/IgfsMetaManager.java        | 53 ++++++++++++++++----
 2 files changed, 45 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06d0e511/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
index 310090d..8c87e97 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
@@ -39,7 +39,7 @@ import java.util.concurrent.locks.ReentrantLock;
  */
 public class IgfsDeleteWorker extends IgfsThread {
     /** Awake frequency, */
-    private static final long FREQUENCY = 1000;
+    private static final long FREQUENCY = 100;
 
     /** How many files/folders to delete at once (i.e in a single transaction). */
     private static final int MAX_DELETE_BATCH = 100;

http://git-wip-us.apache.org/repos/asf/ignite/blob/06d0e511/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 8f1bb69..48ad674 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
@@ -52,14 +53,14 @@ import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaFile
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRenameProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingReplaceProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileReserveSpaceProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileUnlockProcessor;
-import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor;
-import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor;
-import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingReplaceProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePropertiesProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor;
 import org.apache.ignite.internal.util.GridLeanMap;
@@ -152,10 +153,13 @@ public class IgfsMetaManager extends IgfsManager {
     private final boolean client;
 
     /** Compute facade. */
-    private IgniteCompute compute;
+    private volatile IgniteCompute compute;
 
     /** Compute facade for client tasks. */
-    private IgniteCompute metaCompute;
+    private volatile IgniteCompute metaCompute;
+
+    /** Root partition. */
+    private volatile int rootPart = -1;
 
     /**
      * Constructor.
@@ -260,9 +264,14 @@ public class IgfsMetaManager extends IgfsManager {
 
                 return compute().affinityCall(cfg.getMetaCacheName(), affKey, task);
             }
-            else
-                // Otherwise we route to any available data node.
-                return metaCompute().call(task);
+            else {
+                if (metaCache.configuration().getCacheMode() == CacheMode.REPLICATED)
+                    // For replicated cache we do not bother much, just route to any data node.
+                    return metaCompute().call(task);
+                else
+                    // Otherwise we must route request to one of affinity nodes.
+                    return rootCompute().call(task);
+            }
         }
         catch (ClusterTopologyException e) {
             throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e);
@@ -291,7 +300,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Get metadata compute facade for client tasks.
      *
-     * @return Metadata compute facade.
+     * @return Compute facade.
      */
     private IgniteCompute metaCompute() {
         assert client;
@@ -314,6 +323,32 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Get compute facade for root entry affinity nodes.
+     *
+     * @return Compute facade.
+     */
+    private IgniteCompute rootCompute() {
+        IgniteEx ignite = igfsCtx.kernalContext().grid();
+
+        int rootPart0 = rootPart;
+
+        if (rootPart0 == -1) {
+            rootPart0 = metaCache.affinity().partition(IgfsUtils.ROOT_ID);
+
+            rootPart = rootPart0;
+        }
+
+        Collection<ClusterNode> nodes = metaCache.affinity().mapPartitionToPrimaryAndBackups(rootPart0);
+
+        if (nodes.isEmpty())
+            throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes.");
+
+        ClusterGroup cluster = ignite.cluster().forNodes(nodes);
+
+        return ignite.compute(cluster);
+    }
+
+    /**
      * Gets file ID for specified path.
      *
      * @param path Path.


[45/50] ignite git commit: Merge remote-tracking branch 'upstream/gridgain-7.5.27' into gridgain-7.5.27

Posted by vo...@apache.org.
Merge remote-tracking branch 'upstream/gridgain-7.5.27' into gridgain-7.5.27


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

Branch: refs/heads/ignite-3341
Commit: 8a14e471730203db008bd05192d31c11cc1ac172
Parents: 06d0e51 694aabb
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 16:34:14 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 16:34:14 2016 +0300

----------------------------------------------------------------------
 .../zk/TcpDiscoveryZookeeperIpFinder.java        | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[18/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml
new file mode 100644
index 0000000..50b2164
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <bean id="loadBalancingPolicy" class="com.datastax.driver.core.policies.RoundRobinPolicy"/>
+
+    <util:list id="contactPoints" value-type="java.lang.String">
+        <value>cassandra-node-1.abc.com</value>
+        <value>cassandra-node-2.abc.com</value>
+    </util:list>
+
+    <bean id="cassandraAdminDataSource" class="org.apache.ignite.cache.store.cassandra.datasource.DataSource">
+        <property name="user" value="cassandra"/>
+        <property name="password" value="cassandra"/>
+        <property name="contactPoints" ref="contactPoints"/>
+        <property name="readConsistency" value="ONE"/>
+        <property name="writeConsistency" value="ONE"/>
+        <property name="loadBalancingPolicy" ref="loadBalancingPolicy"/>
+    </bean>
+
+    <bean id="cache1_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="primitive_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" column="key"/>
+    <valuePersistence class="java.lang.Long" strategy="PRIMITIVE" column="value"/>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- Turn on client mode -->
+        <property name="clientMode" value="true"/>
+
+        <property name="metricsLogFrequency" value="0"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Configuring remote ignite cluster connections -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>ignite-node-1</value>
+                                <value>ignite-node-2</value>
+                                <value>ignite-node-3</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml
new file mode 100644
index 0000000..1dc6f8a
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml
@@ -0,0 +1,106 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <!-- Cassandra connection settings -->
+    <bean id="loadBalancingPolicy" class="com.datastax.driver.core.policies.RoundRobinPolicy"/>
+
+    <util:list id="contactPoints" value-type="java.lang.String">
+        <value>cassandra-node-1.abc.com</value>
+        <value>cassandra-node-2.abc.com</value>
+    </util:list>
+
+    <bean id="cassandraAdminDataSource" class="org.apache.ignite.cache.store.cassandra.datasource.DataSource">
+        <property name="user" value="cassandra"/>
+        <property name="password" value="cassandra"/>
+        <property name="contactPoints" ref="contactPoints"/>
+        <property name="readConsistency" value="ONE"/>
+        <property name="writeConsistency" value="ONE"/>
+        <property name="loadBalancingPolicy" ref="loadBalancingPolicy"/>
+    </bean>
+
+    <!-- Persistence settings for 'cache1' -->
+    <bean id="cache1_persistence_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="primitive_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" column="key"/>
+    <valuePersistence class="java.lang.Long" strategy="PRIMITIVE" column="value"/>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+	
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Partitioned cache example configuration (Atomic mode). -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+			
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery. For information on all options refer
+                        to our documentation: http://apacheignite.readme.io/docs/cluster-config
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml
new file mode 100644
index 0000000..27882e6
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml
@@ -0,0 +1,21 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="primitive_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" column="key"/>
+    <valuePersistence class="java.lang.Long" strategy="PRIMITIVE" column="value"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml
new file mode 100644
index 0000000..62c1f4a
--- /dev/null
+++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml
@@ -0,0 +1,21 @@
+<!--
+  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.
+-->
+
+<persistence keyspace="test1" table="primitive_test2">
+    <keyPersistence class="java.lang.String" strategy="PRIMITIVE" column="key"/>
+    <valuePersistence class="java.lang.String" strategy="PRIMITIVE" column="value"/>
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/resources/tests.properties
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/resources/tests.properties b/modules/cassandra/src/test/resources/tests.properties
new file mode 100644
index 0000000..2c91e57
--- /dev/null
+++ b/modules/cassandra/src/test/resources/tests.properties
@@ -0,0 +1,50 @@
+# 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.
+
+# Number of elements for CacheStore bulk operations: loadAll, writeAll, deleteAll
+bulk.operation.size=100
+
+# ----- Load tests settings -----
+
+# Ignite cache to be used by load tests
+load.tests.cache.name=cache1
+#load.tests.cache.name=cache3
+
+# Number of simultaneous threads for each load test
+load.tests.threads.count=10
+
+# Warm up period (in milliseconds) for each load test before starting any measurements
+load.tests.warmup.period=180000
+
+# Time for each load test execution excluding warm up period (in milliseconds)
+load.tests.execution.time=300000
+
+# Latency (in milliseconds) between two sequential requests to Cassandra/Ignite
+load.tests.requests.latency=0
+
+# Resource specifying persistence settings for all load tests
+load.tests.persistence.settings=org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml
+#load.tests.persistence.settings=org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml
+
+# Resource specifying Ignite configuration for all load tests
+load.tests.ignite.config=org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml
+
+# Key generator for load tests
+load.tests.key.generator=org.apache.ignite.tests.load.LongGenerator
+#load.tests.key.generator=org.apache.ignite.tests.load.PersonIdGenerator
+
+# Value generator for load tests
+load.tests.value.generator=org.apache.ignite.tests.load.LongGenerator
+#load.tests.value.generator=org.apache.ignite.tests.load.PersonGenerator

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/scripts/cassandra-load-tests.bat
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/scripts/cassandra-load-tests.bat b/modules/cassandra/src/test/scripts/cassandra-load-tests.bat
new file mode 100644
index 0000000..c64de1e
--- /dev/null
+++ b/modules/cassandra/src/test/scripts/cassandra-load-tests.bat
@@ -0,0 +1,41 @@
+::
+:: 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.
+::
+
+echo off
+
+echo.
+
+set TESTS_CLASSPATH="%~dp0lib\*;%~dp0settings"
+
+call %~dp0jvm-opts.bat %*
+
+call java %JVM_OPTS% -cp "%TESTS_CLASSPATH%" "org.apache.ignite.tests.CassandraDirectPersistenceLoadTest"
+
+if %errorLevel% NEQ 0 (
+    echo.
+    echo --------------------------------------------------------------------------------
+    echo [ERROR] Tests execution failed
+    echo --------------------------------------------------------------------------------
+    echo.
+    exit /b %errorLevel%
+)
+
+echo.
+echo --------------------------------------------------------------------------------
+echo [INFO] Tests execution succeed
+echo --------------------------------------------------------------------------------
+echo.

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/scripts/cassandra-load-tests.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/scripts/cassandra-load-tests.sh b/modules/cassandra/src/test/scripts/cassandra-load-tests.sh
new file mode 100644
index 0000000..dda25dc
--- /dev/null
+++ b/modules/cassandra/src/test/scripts/cassandra-load-tests.sh
@@ -0,0 +1,39 @@
+#!/bin/sh
+#
+# 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.
+#
+
+TESTS_ROOT=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ))
+TESTS_CLASSPATH="$TESTS_ROOT/lib/*:$TESTS_ROOT/settings"
+
+. $TESTS_ROOT/jvm-opt.sh $@
+
+java $JVM_OPTS -cp "$TESTS_CLASSPATH" "org.apache.ignite.tests.CassandraDirectPersistenceLoadTest"
+
+if [ $? -ne 0 ]; then
+    echo
+    echo "--------------------------------------------------------------------------------"
+    echo "[ERROR] Tests execution failed"
+    echo "--------------------------------------------------------------------------------"
+    echo
+    exit 1
+fi
+
+echo
+echo "--------------------------------------------------------------------------------"
+echo "[INFO] Tests execution succeed"
+echo "--------------------------------------------------------------------------------"
+echo

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/scripts/ignite-load-tests.bat
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/scripts/ignite-load-tests.bat b/modules/cassandra/src/test/scripts/ignite-load-tests.bat
new file mode 100644
index 0000000..5a45ffc
--- /dev/null
+++ b/modules/cassandra/src/test/scripts/ignite-load-tests.bat
@@ -0,0 +1,41 @@
+::
+:: 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.
+::
+
+echo off
+
+echo.
+
+set TESTS_CLASSPATH="%~dp0\lib*;%~dp0settings"
+
+call %~dp0jvm-opts.bat %*
+
+call java %JVM_OPTS% -cp "%TESTS_CLASSPATH%" "org.apache.ignite.tests.IgnitePersistentStoreLoadTest"
+
+if %errorLevel% NEQ 0 (
+    echo.
+    echo --------------------------------------------------------------------------------
+    echo [ERROR] Tests execution failed
+    echo --------------------------------------------------------------------------------
+    echo.
+    exit /b %errorLevel%
+)
+
+echo.
+echo --------------------------------------------------------------------------------
+echo [INFO] Tests execution succeed
+echo --------------------------------------------------------------------------------
+echo.

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/scripts/ignite-load-tests.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/scripts/ignite-load-tests.sh b/modules/cassandra/src/test/scripts/ignite-load-tests.sh
new file mode 100644
index 0000000..c2e1a41
--- /dev/null
+++ b/modules/cassandra/src/test/scripts/ignite-load-tests.sh
@@ -0,0 +1,39 @@
+#!/bin/sh
+#
+# 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.
+#
+
+TESTS_ROOT=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ))
+TESTS_CLASSPATH="$TESTS_ROOT/lib/*:$TESTS_ROOT/settings"
+
+. $TESTS_ROOT/jvm-opt.sh $@
+
+java $JVM_OPTS -cp "$TESTS_CLASSPATH" "org.apache.ignite.tests.IgnitePersistentStoreLoadTest"
+
+if [ $? -ne 0 ]; then
+    echo
+    echo "--------------------------------------------------------------------------------"
+    echo "[ERROR] Tests execution failed"
+    echo "--------------------------------------------------------------------------------"
+    echo
+    exit 1
+fi
+
+echo
+echo "--------------------------------------------------------------------------------"
+echo "[INFO] Tests execution succeed"
+echo "--------------------------------------------------------------------------------"
+echo

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/scripts/jvm-opt.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/scripts/jvm-opt.sh b/modules/cassandra/src/test/scripts/jvm-opt.sh
new file mode 100644
index 0000000..d4e70f0
--- /dev/null
+++ b/modules/cassandra/src/test/scripts/jvm-opt.sh
@@ -0,0 +1,21 @@
+#!/bin/sh
+#
+# 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.
+#
+
+JVM_OPTS="-Xms6g -Xmx6g -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m"
+JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+UseTLAB -XX:NewSize=128m -XX:MaxNewSize=768m"
+JVM_OPTS="$JVM_OPTS -Xss16m"

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/scripts/jvm-opts.bat
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/scripts/jvm-opts.bat b/modules/cassandra/src/test/scripts/jvm-opts.bat
new file mode 100644
index 0000000..1937efb
--- /dev/null
+++ b/modules/cassandra/src/test/scripts/jvm-opts.bat
@@ -0,0 +1,24 @@
+::
+:: 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.
+::
+
+echo off
+
+echo.
+
+set "JVM_OPTS=-Xms6g -Xmx6g -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m"
+set "JVM_OPTS=%JVM_OPTS% -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+UseTLAB -XX:NewSize=128m -XX:MaxNewSize=768m"
+set "JVM_OPTS=%JVM_OPTS% -Xss16m"

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b8aaa5f..0761aec 100644
--- a/pom.xml
+++ b/pom.xml
@@ -84,6 +84,7 @@
         <module>modules/osgi-karaf</module>
         <module>modules/osgi</module>
         <module>modules/web/ignite-weblogic-test</module>
+        <module>modules/cassandra</module>
     </modules>
 
     <profiles>
@@ -322,6 +323,14 @@
 
                                         <chmod dir="${basedir}/target/release-package" perm="755" includes="**/*.sh" />
 
+                                        <!--
+                                        Line ending bugfix for builds packaged in MS Windows machine to adjust Linux based
+                                        end-of-line characters for Linux shell scripts
+                                        -->
+                                        <fixcrlf srcdir="${basedir}/target/release-package" eol="lf" eof="remove">
+                                            <include name="**/*.sh" />
+                                        </fixcrlf>
+
                                         <zip destfile="${basedir}/target/bin/${ignite.zip.pattern}.zip" encoding="UTF-8">
                                             <zipfileset dir="${basedir}/target/release-package" prefix="${ignite.zip.pattern}" filemode="755">
                                                 <include name="**/*.sh" />


[04/50] ignite git commit: ignite-114 Load value from store for cache 'invoke'

Posted by vo...@apache.org.
ignite-114 Load value from store for cache 'invoke'


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

Branch: refs/heads/ignite-3341
Commit: e10ffef00e91f88932c6478741eb00abe8992de4
Parents: 2b64e7c
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 14 10:05:20 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 14 10:05:20 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  34 +-
 .../processors/cache/GridCacheEntryEx.java      |   9 +-
 .../processors/cache/GridCacheMapEntry.java     | 100 +++--
 .../processors/cache/GridCacheUtils.java        |   3 +
 .../distributed/dht/GridDhtCacheAdapter.java    |  10 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |  19 +-
 .../dht/GridPartitionedGetFuture.java           |   2 -
 .../dht/GridPartitionedSingleGetFuture.java     |   2 -
 .../dht/atomic/GridDhtAtomicCache.java          |   8 -
 .../dht/colocated/GridDhtColocatedCache.java    |   4 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   3 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   3 +-
 .../distributed/near/GridNearGetFuture.java     |   4 -
 .../cache/distributed/near/GridNearTxLocal.java |   7 +-
 .../local/atomic/GridLocalAtomicCache.java      |   8 -
 .../cache/transactions/IgniteTxAdapter.java     |   2 -
 .../cache/transactions/IgniteTxEntry.java       |  24 +-
 .../cache/transactions/IgniteTxHandler.java     |  46 +++
 .../transactions/IgniteTxLocalAdapter.java      |  34 +-
 .../datastreamer/DataStreamerImpl.java          |   3 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   5 +-
 .../cache/IgniteCacheAbstractTest.java          |   2 +-
 ...gniteCacheInvokeReadThroughAbstractTest.java | 382 +++++++++++++++++++
 ...iteCacheInvokeReadThroughSingleNodeTest.java | 106 +++++
 .../cache/IgniteCacheInvokeReadThroughTest.java | 182 +++++----
 .../IgniteCacheReadThroughStoreCallTest.java    | 288 ++++++++++++++
 .../IgniteCacheLoaderWriterAbstractTest.java    |  10 +
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 30 files changed, 1110 insertions(+), 214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 6ccb506..6e647c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -3655,8 +3655,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         GridCacheEntryEx entry = entryEx(key, false);
 
         try {
-            entry.initialValue(cacheVal, ver, ttl, CU.EXPIRE_TIME_CALCULATE, false, topVer,
-                replicate ? DR_LOAD : DR_NONE);
+            entry.initialValue(cacheVal,
+                ver,
+                ttl,
+                CU.EXPIRE_TIME_CALCULATE,
+                false,
+                topVer,
+                replicate ? DR_LOAD : DR_NONE,
+                true);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException("Failed to put cache value: " + entry, e);
@@ -5102,19 +5108,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         throws IgniteCheckedException, GridCacheEntryRemovedException
     {
         CacheObject val = entry.innerGet(
-            null,
-            null,
-            false,
-            false,
-            false,
-            true,
-            false,
-            false,
-            false,
-            null,
-            null,
-            null,
-            null,
+            /*ver*/null,
+            /*tx*/null,
+            /*swap*/false,
+            /*readThrough*/false,
+            /*metrics*/false,
+            /*evt*/false,
+            /*tmp*/false,
+            /*subjId*/null,
+            /*transformClo*/null,
+            /*taskName*/null,
+            /*expiryPlc*/null,
             !deserializeBinary);
 
         if (val == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 31bd887..c5fdde6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -278,9 +278,6 @@ public interface GridCacheEntryEx {
      * @param tx Ongoing transaction (possibly null).
      * @param readSwap Flag indicating whether to check swap memory.
      * @param readThrough Flag indicating whether to read through.
-     * @param failFast If {@code true}, then throw {@link GridCacheFilterFailedException} if
-     *      filter didn't pass.
-     * @param unmarshal Unmarshal flag.
      * @param updateMetrics If {@code true} then metrics should be updated.
      * @param evt Flag to signal event notification.
      * @param tmp If {@code true} can return temporary instance which is valid while entry lock is held,
@@ -299,8 +296,6 @@ public interface GridCacheEntryEx {
         @Nullable IgniteInternalTx tx,
         boolean readSwap,
         boolean readThrough,
-        boolean failFast,
-        boolean unmarshal,
         boolean updateMetrics,
         boolean evt,
         boolean tmp,
@@ -677,6 +672,7 @@ public interface GridCacheEntryEx {
      * @param preload Flag indicating whether entry is being preloaded.
      * @param topVer Topology version.
      * @param drType DR type.
+     * @param fromStore {@code True} if value was loaded from store.
      * @return {@code True} if initial value was set.
      * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
@@ -687,7 +683,8 @@ public interface GridCacheEntryEx {
         long expireTime,
         boolean preload,
         AffinityTopologyVersion topVer,
-        GridDrType drType) throws IgniteCheckedException, GridCacheEntryRemovedException;
+        GridDrType drType,
+        boolean fromStore) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Sets new value if current version is <tt>0</tt> using swap entry data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index f442202..b5359f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -750,8 +750,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         @Nullable IgniteInternalTx tx,
         boolean readSwap,
         boolean readThrough,
-        boolean failFast,
-        boolean unmarshal,
         boolean updateMetrics,
         boolean evt,
         boolean tmp,
@@ -766,7 +764,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             readSwap,
             readThrough,
             evt,
-            unmarshal,
             updateMetrics,
             tmp,
             subjId,
@@ -796,7 +793,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             readSwap,
             false,
             evt,
-            unmarshal,
             updateMetrics,
             false,
             subjId,
@@ -815,7 +811,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         boolean readSwap,
         boolean readThrough,
         boolean evt,
-        boolean unmarshal,
         boolean updateMetrics,
         boolean tmp,
         UUID subjId,
@@ -1985,6 +1980,51 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (isStartVersion())
                 unswap(retval, false);
 
+            // Prepare old value.
+            oldVal = needVal ? rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : val;
+
+            // Possibly read value from store.
+            boolean readFromStore = false;
+
+            Object old0 = null;
+
+            if (readThrough && needVal && oldVal == null && (cctx.readThrough() &&
+                (op == GridCacheOperation.TRANSFORM || cctx.loadPreviousValue()))) {
+                old0 = readThrough(null, key, false, subjId, taskName);
+
+                oldVal = cctx.toCacheObject(old0);
+
+                readFromStore = true;
+
+                // Detach value before index update.
+                oldVal = cctx.kernalContext().cacheObjects().prepareForCache(oldVal, cctx);
+
+                // Calculate initial TTL and expire time.
+                long initTtl;
+                long initExpireTime;
+
+                if (expiryPlc != null && oldVal != null) {
+                    IgniteBiTuple<Long, Long> initTtlAndExpireTime = initialTtlAndExpireTime(expiryPlc);
+
+                    initTtl = initTtlAndExpireTime.get1();
+                    initExpireTime = initTtlAndExpireTime.get2();
+                }
+                else {
+                    initTtl = CU.TTL_ETERNAL;
+                    initExpireTime = CU.EXPIRE_TIME_ETERNAL;
+                }
+
+                if (oldVal != null)
+                    updateIndex(oldVal, initExpireTime, ver, null);
+                else
+                    clearIndex(null);
+
+                update(oldVal, initExpireTime, initTtl, ver, true);
+
+                if (deletedUnlocked() && oldVal != null && !isInternal())
+                    deletedUnlocked(false);
+            }
+
             Object transformClo = null;
 
             // Request-level conflict resolution is needed, i.e. we do not know who will win in advance.
@@ -2192,51 +2232,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         "Invalid version for inner update [isNew=" + isNew() + ", entry=" + this + ", newVer=" + newVer + ']';
             }
 
-            // Prepare old value and value bytes.
-            oldVal = needVal ? rawGetOrUnmarshalUnlocked(!retval && !isOffHeapValuesOnly()) : val;
-
-            // Possibly read value from store.
-            boolean readFromStore = false;
-
-            Object old0 = null;
-
-            if (readThrough && needVal && oldVal == null && (cctx.readThrough() &&
-                (op == GridCacheOperation.TRANSFORM || cctx.loadPreviousValue()))) {
-                old0 = readThrough(null, key, false, subjId, taskName);
-
-                oldVal = cctx.toCacheObject(old0);
-
-                readFromStore = true;
-
-                // Detach value before index update.
-                oldVal = cctx.kernalContext().cacheObjects().prepareForCache(oldVal, cctx);
-
-                // Calculate initial TTL and expire time.
-                long initTtl;
-                long initExpireTime;
-
-                if (expiryPlc != null && oldVal != null) {
-                    IgniteBiTuple<Long, Long> initTtlAndExpireTime = initialTtlAndExpireTime(expiryPlc);
-
-                    initTtl = initTtlAndExpireTime.get1();
-                    initExpireTime = initTtlAndExpireTime.get2();
-                }
-                else {
-                    initTtl = CU.TTL_ETERNAL;
-                    initExpireTime = CU.EXPIRE_TIME_ETERNAL;
-                }
-
-                if (oldVal != null)
-                    updateIndex(oldVal, initExpireTime, ver, null);
-                else
-                    clearIndex(null);
-
-                update(oldVal, initExpireTime, initTtl, ver, true);
-
-                if (deletedUnlocked() && oldVal != null && !isInternal())
-                    deletedUnlocked(false);
-            }
-
             // Apply metrics.
             if (metrics && cctx.cache().configuration().isStatisticsEnabled() && needVal) {
                 // PutIfAbsent methods mustn't update hit/miss statistics
@@ -3435,7 +3430,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         long expireTime,
         boolean preload,
         AffinityTopologyVersion topVer,
-        GridDrType drType)
+        GridDrType drType,
+        boolean fromStore)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         synchronized (this) {
             checkObsolete();
@@ -3488,7 +3484,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                     cctx.dataStructures().onEntryUpdated(key, false, true);
                 }
 
-                if (cctx.store().isLocal()) {
+                if (!fromStore && cctx.store().isLocal()) {
                     if (val != null)
                         cctx.store().put(null, key, val, ver);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 6ce5735..87c4a3a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -155,6 +155,9 @@ public class GridCacheUtils {
     /** Keep serialized flag. */
     public static final int KEEP_BINARY_FLAG_MASK = 0x2;
 
+    /** Flag indicating that old value for 'invoke' operation was non null on primary node. */
+    public static final int OLD_VAL_ON_PRIMARY = 0x4;
+
     /** Empty predicate array. */
     private static final IgnitePredicate[] EMPTY = new IgnitePredicate[0];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 5cc079d..ed1996a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -564,8 +564,14 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
                     entry = entryEx(key, false);
 
-                    entry.initialValue(cacheVal, ver, ttl, CU.EXPIRE_TIME_CALCULATE, false, topVer,
-                        replicate ? DR_LOAD : DR_NONE);
+                    entry.initialValue(cacheVal,
+                        ver,
+                        ttl,
+                        CU.EXPIRE_TIME_CALCULATE,
+                        false,
+                        topVer,
+                        replicate ? DR_LOAD : DR_NONE,
+                        false);
                 }
                 catch (IgniteCheckedException e) {
                     throw new IgniteException("Failed to put cache value: " + entry, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 5659436..0a3513a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -1053,7 +1053,14 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                             try {
                                 CacheObject val0 = cctx.toCacheObject(val);
 
-                                entry0.initialValue(val0, ver, 0, 0, false, topVer, GridDrType.DR_LOAD);
+                                entry0.initialValue(val0,
+                                    ver,
+                                    0,
+                                    0,
+                                    false,
+                                    topVer,
+                                    GridDrType.DR_LOAD,
+                                    true);
                             }
                             catch (GridCacheEntryRemovedException e) {
                                 assert false : "Should not get removed exception while holding lock on entry " +
@@ -1210,8 +1217,13 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                     try {
                         GridCacheEntryEx entry = cctx.cache().entryEx(info.key(), topVer);
 
-                        if (entry.initialValue(info.value(), info.version(), info.ttl(),
-                            info.expireTime(), true, topVer, replicate ? DR_PRELOAD : DR_NONE)) {
+                        if (entry.initialValue(info.value(),
+                            info.version(),
+                            info.ttl(),
+                            info.expireTime(),
+                            true, topVer,
+                            replicate ? DR_PRELOAD : DR_NONE,
+                            false)) {
                             if (rec && !entry.isInternal())
                                 cctx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(),
                                     (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_LOADED, info.value(), true, null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index b2da39c..0ca02c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -1181,8 +1181,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                         tx,
                                         /*swap*/true,
                                         /*read-through*/false,
-                                        /*fail-fast.*/false,
-                                        /*unmarshal*/false,
                                         /*update-metrics*/true,
                                         /*event notification*/req.returnValue(i),
                                         /*temporary*/false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index e17cbbc..cedfee5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -333,9 +333,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 if (hasFilters || retVal || txEntry.op() == DELETE || txEntry.op() == TRANSFORM) {
                     cached.unswap(retVal);
 
-                    boolean readThrough = (retVal || hasFilters) &&
-                        cacheCtx.config().isLoadPreviousValue() &&
-                        !txEntry.skipStore();
+                    boolean readThrough = !txEntry.skipStore() &&
+                        (txEntry.op() == TRANSFORM || ((retVal || hasFilters) && cacheCtx.config().isLoadPreviousValue()));
 
                     boolean evt = retVal || txEntry.op() == TRANSFORM;
 
@@ -351,8 +350,6 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         tx,
                         /*swap*/true,
                         readThrough,
-                        /*fail fast*/false,
-                        /*unmarshal*/true,
                         /*metrics*/retVal,
                         /*event*/evt,
                         /*tmp*/false,
@@ -376,6 +373,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                             boolean modified = false;
 
+                            txEntry.oldValueOnPrimary(val != null);
+
                             for (T2<EntryProcessor<Object, Object, Object>, Object[]> t : txEntry.entryProcessors()) {
                                  CacheInvokeEntry<Object, Object> invokeEntry = new CacheInvokeEntry<>(key, val,
                                      txEntry.cached().version(), keepBinary, txEntry.cached());
@@ -1559,8 +1558,14 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         GridDrType drType = cacheCtx.isDrEnabled() ? GridDrType.DR_PRELOAD : GridDrType.DR_NONE;
 
                         try {
-                            if (entry.initialValue(info.value(), info.version(),
-                                info.ttl(), info.expireTime(), true, topVer, drType)) {
+                            if (entry.initialValue(info.value(),
+                                info.version(),
+                                info.ttl(),
+                                info.expireTime(),
+                                true,
+                                topVer,
+                                drType,
+                                false)) {
                                 if (rec && !entry.isInternal())
                                     cacheCtx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(),
                                         (IgniteUuid)null, null, EVT_CACHE_REBALANCE_OBJECT_LOADED, info.value(), true, null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 7d2c5db..b3a485c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -473,8 +473,6 @@ public class GridPartitionedGetFuture<K, V> extends CacheDistributedGetFutureAda
                             null,
                             /*swap*/true,
                             /*read-through*/false,
-                            /*fail-fast*/true,
-                            /*unmarshal*/true,
                             /**update-metrics*/false,
                             /*event*/!skipVals,
                             /*temporary*/false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index b0b9d7c..7788b63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -397,8 +397,6 @@ public class GridPartitionedSingleGetFuture extends GridFutureAdapter<Object> im
                             null,
                             /*swap*/true,
                             /*read-through*/false,
-                            /*fail-fast*/true,
-                            /*unmarshal*/true,
                             /**update-metrics*/false,
                             /*event*/!skipVals,
                             /*temporary*/false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 08f1f30..ad87add 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1219,8 +1219,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                     null,
                                     /*swap*/true,
                                     /*read-through*/false,
-                                    /*fail-fast*/true,
-                                    /*unmarshal*/true,
                                     /**update-metrics*/false,
                                     /*event*/!skipVals,
                                     /*temporary*/false,
@@ -1695,8 +1693,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         null,
                         /*read swap*/true,
                         /*read through*/true,
-                        /*fail fast*/false,
-                        /*unmarshal*/true,
                         /*metrics*/true,
                         /*event*/true,
                         /*temporary*/true,
@@ -1855,8 +1851,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                              null,
                             /*read swap*/true,
                             /*read through*/ctx.loadPreviousValue(),
-                            /*fail fast*/false,
-                            /*unmarshal*/true,
                             /*metrics*/true,
                             /*event*/true,
                             /*temporary*/true,
@@ -1902,8 +1896,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                             null,
                             /*read swap*/true,
                             /*read through*/ctx.loadPreviousValue(),
-                            /*fail fast*/false,
-                            /*unmarshal*/true,
                             /*metrics*/true,
                             /*event*/true,
                             /*temporary*/true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 35831a3..e8e8298 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -507,8 +507,6 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                                     null,
                                     /*swap*/true,
                                     /*read-through*/false,
-                                    /*fail-fast*/true,
-                                    /*unmarshal*/true,
                                     /**update-metrics*/false,
                                     /*event*/!skipVals,
                                     /*temporary*/false,
@@ -1057,7 +1055,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
 
             IgniteInternalFuture<GridCacheReturn> txFut = tx.lockAllAsync(cacheCtx,
                 keys,
-                tx.implicit(),
+                retval,
                 txRead,
                 accessTtl,
                 skipStore,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 4da1f38..7cbd77c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -552,7 +552,8 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                             info.expireTime(),
                             true,
                             topVer,
-                            replicate ? DR_PRELOAD : DR_NONE
+                            replicate ? DR_PRELOAD : DR_NONE,
+                            false
                         )) {
                             if (rec && !entry.isInternal())
                                 cctx.events().addEvent(entry.partition(), entry.key(), cctx.localNodeId(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/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 6181d9e..6fde827 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
@@ -689,7 +689,8 @@ public class GridDhtPartitionDemander {
                         entry.expireTime(),
                         true,
                         topVer,
-                        cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE
+                        cctx.isDrEnabled() ? DR_PRELOAD : DR_NONE,
+                        false
                     )) {
                         cctx.evicts().touch(cached, topVer); // Start tracking.
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index edac92c..73b9d38 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -457,8 +457,6 @@ public final class GridNearGetFuture<K, V> extends CacheDistributedGetFutureAdap
                             tx,
                             /*swap*/false,
                             /*read-through*/false,
-                            /*fail-fast*/true,
-                            /*unmarshal*/true,
                             /*metrics*/true,
                             /*events*/!skipVals,
                             /*temporary*/false,
@@ -599,8 +597,6 @@ public final class GridNearGetFuture<K, V> extends CacheDistributedGetFutureAdap
                             tx,
                             /*swap*/true,
                             /*read-through*/false,
-                            /*fail-fast*/true,
-                            /*unmarshal*/true,
                             /*update-metrics*/false,
                             /*events*/!nearRead && !skipVals,
                             /*temporary*/false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 3ff165b..bea2ce5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -1138,16 +1138,17 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     /**
      * @param cacheCtx Cache context.
      * @param keys Keys.
-     * @param implicit Implicit flag.
+     * @param retval Return value flag.
      * @param read Read flag.
      * @param accessTtl Access ttl.
      * @param <K> Key type.
      * @param skipStore Skip store flag.
+     * @param keepBinary Keep binary flag.
      * @return Future with respond.
      */
     public <K> IgniteInternalFuture<GridCacheReturn> lockAllAsync(GridCacheContext cacheCtx,
         final Collection<? extends K> keys,
-        boolean implicit,
+        boolean retval,
         boolean read,
         long accessTtl,
         boolean skipStore,
@@ -1181,7 +1182,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
             this,
             isInvalidate(),
             read,
-            /*retval*/false,
+            retval,
             isolation,
             accessTtl,
             CU.empty0(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index d2d8d62..7caea23 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -628,8 +628,6 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                                 null,
                                 /*swap*/swapOrOffheap,
                                 /*read-through*/false,
-                                /*fail-fast*/false,
-                                /*unmarshal*/true,
                                 /**update-metrics*/true,
                                 /**event*/!skipVals,
                                 /**temporary*/false,
@@ -1187,8 +1185,6 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                             null,
                             /*swap*/true,
                             /*read-through*/true,
-                            /*fail-fast*/false,
-                            /*unmarshal*/true,
                             /**update-metrics*/true,
                             /**event*/true,
                             /**temporary*/true,
@@ -1311,8 +1307,6 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                                 null,
                                 /*swap*/true,
                                 /*read-through*/ctx.loadPreviousValue(),
-                                /*fail-fast*/false,
-                                /*unmarshal*/true,
                                 /**update-metrics*/true,
                                 /**event*/true,
                                 /**temporary*/true,
@@ -1348,8 +1342,6 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                                 null,
                                 /*swap*/true,
                                 /*read-through*/ctx.loadPreviousValue(),
-                                /*fail-fast*/false,
-                                /*unmarshal*/true,
                                 /**update-metrics*/true,
                                 /**event*/true,
                                 /**temporary*/true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/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 66d4df0..3bb3a17 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
@@ -1492,8 +1492,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                     this,
                     /*swap*/false,
                     /*read through*/false,
-                    /*fail fast*/true,
-                    /*unmarshal*/true,
                     /*metrics*/metrics,
                     /*event*/recordEvt,
                     /*temporary*/true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 90aa8d2..3258ce9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -57,6 +58,7 @@ import org.jetbrains.annotations.Nullable;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.KEEP_BINARY_FLAG_MASK;
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.OLD_VAL_ON_PRIMARY;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.SKIP_STORE_FLAG_MASK;
 
 /**
@@ -185,9 +187,11 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     private byte[] expiryPlcBytes;
 
     /**
-     * Additional flags.
-     * GridCacheUtils.SKIP_STORE_FLAG_MASK - for skipStore flag value.
-     * GridCacheUtils.KEEP_BINARY_FLAG_MASK - for withKeepBinary flag.
+     * Additional flags:
+     * <ul>
+     * <li>{@link GridCacheUtils#SKIP_STORE_FLAG_MASK} - for skipStore flag value.</li>
+     * <li>{@link GridCacheUtils#KEEP_BINARY_FLAG_MASK} - for withKeepBinary flag.</li>
+     * </ul>
      */
     private byte flags;
 
@@ -483,6 +487,20 @@ public class IgniteTxEntry implements GridPeerDeployAware, Message {
     }
 
     /**
+     * @param oldValOnPrimary {@code True} If old value for 'invoke' operation was non null on primary node.
+     */
+    public void oldValueOnPrimary(boolean oldValOnPrimary) {
+        setFlag(oldValOnPrimary, OLD_VAL_ON_PRIMARY);
+    }
+
+    /**
+     * @return {@code True} If old value for 'invoke' operation was non null on primary node.
+     */
+    public boolean oldValueOnPrimary() {
+        return isFlag(OLD_VAL_ON_PRIMARY);
+    }
+
+    /**
      * Sets keep binary flag value.
      *
      * @param keepBinary Keep binary flag value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index b25baf8..5b09760 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -25,9 +25,11 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
@@ -73,6 +75,7 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled;
 import static org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture.FINISH_NEAR_ONE_PHASE_SINCE;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.USER_FINISH;
@@ -1165,6 +1168,49 @@ public class IgniteTxHandler {
                             if (info != null && !info.isNew() && !info.isDeleted())
                                 res.addPreloadEntry(info);
                         }
+
+                        if (cacheCtx.readThroughConfigured() &&
+                            !entry.skipStore() &&
+                            entry.op() == TRANSFORM &&
+                            entry.oldValueOnPrimary() &&
+                            !entry.hasValue()) {
+                            while (true) {
+                                try {
+                                    GridCacheEntryEx cached = entry.cached();
+
+                                    if (cached == null)
+                                        cached = cacheCtx.cache().entryEx(entry.key(), req.topologyVersion());
+
+                                    CacheObject val = cached.innerGet(
+                                        /*ver*/null,
+                                        tx,
+                                        /*readSwap*/true,
+                                        /*readThrough*/false,
+                                        /*updateMetrics*/false,
+                                        /*evt*/false,
+                                        /*tmp*/false,
+                                        tx.subjectId(),
+                                        /*transformClo*/null,
+                                        tx.resolveTaskName(),
+                                        /*expiryPlc*/null,
+                                        /*keepBinary*/true);
+
+                                    if (val == null)
+                                        val = cacheCtx.toCacheObject(cacheCtx.store().load(null, entry.key()));
+
+                                    if (val != null)
+                                        entry.readValue(val);
+
+                                    break;
+                                }
+                                catch (GridCacheEntryRemovedException e) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Got entry removed exception, will retry: " + entry.txKey());
+
+                                    entry.cached(null);
+                                }
+                            }
+                        }
                     }
                     catch (GridDhtInvalidPartitionException e) {
                         tx.addInvalidPartition(cacheCtx, e.partition());

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/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 8eea8dd..28ecda5 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
@@ -1255,8 +1255,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     this,
                                     /*swap*/true,
                                     /*read-through*/false,
-                                    /*fail fast*/true,
-                                    /*unmarshal*/true,
                                     /*metrics*/true,
                                     /*event*/!skipVals,
                                     /*temporary*/false,
@@ -1340,9 +1338,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     null,
                                     this,
                                     /*swap*/true,
-                                    /*no read-through*/false,
-                                    /*fail-fast*/true,
-                                    /*unmarshal*/true,
+                                    /*read-through*/false,
                                     /*metrics*/true,
                                     /*event*/true,
                                     /*temporary*/false,
@@ -1687,8 +1683,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                             IgniteTxLocalAdapter.this,
                                             cacheCtx.isSwapOrOffheapEnabled(),
                                             /*read-through*/false,
-                                            /*fail-fast*/true,
-                                            /*unmarshal*/true,
                                             /*metrics*/true,
                                             /*events*/!skipVals,
                                             /*temporary*/false,
@@ -1992,7 +1986,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     needReadVer,
                     singleRmv,
                     hasFilters,
-                    skipStore,
+                    /*read through*/(entryProcessor != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore,
                     retval,
                     keepBinary);
             }
@@ -2153,7 +2147,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     needReadVer,
                     singleRmv,
                     hasFilters,
-                    skipStore,
+                    /*read through*/(invokeMap != null || cacheCtx.config().isLoadPreviousValue()) && !skipStore,
                     retval,
                     keepBinary);
             }
@@ -2173,7 +2167,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
      * @param needReadVer Read version flag.
      * @param singleRmv {@code True} for single remove operation.
      * @param hasFilters {@code True} if filters not empty.
-     * @param skipStore Skip store flag.
+     * @param readThrough Read through flag.
      * @param retval Return value flag.
      * @return Load future.
      */
@@ -2185,7 +2179,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         final boolean needReadVer,
         final boolean singleRmv,
         final boolean hasFilters,
-        final boolean skipStore,
+        final boolean readThrough,
         final boolean retval,
         final boolean keepBinary) {
         GridInClosure3<KeyCacheObject, Object, GridCacheVersion> c =
@@ -2218,6 +2212,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         if (e.op() == TRANSFORM) {
                             GridCacheVersion ver;
 
+                            e.readValue(cacheVal);
+
                             try {
                                 ver = e.cached().version();
                             }
@@ -2243,7 +2239,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
         return loadMissing(
             cacheCtx,
-            /*read through*/cacheCtx.config().isLoadPreviousValue() && !skipStore,
+            readThrough,
             /*async*/true,
             keys,
             /*skipVals*/singleRmv,
@@ -2354,8 +2350,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     this,
                                     /*swap*/false,
                                     /*read-through*/false,
-                                    /*fail-fast*/false,
-                                    /*unmarshal*/retval,
                                     /*metrics*/retval,
                                     /*events*/retval,
                                     /*temporary*/false,
@@ -2652,16 +2646,16 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     if (retval || invoke) {
                         if (!cacheCtx.isNear()) {
                             if (!hasPrevVal) {
-                                boolean readThrough =
-                                    (invoke || cacheCtx.loadPreviousValue()) && !txEntry.skipStore();
+                                // For non-local cache should read from store after lock on primary.
+                                boolean readThrough = cacheCtx.isLocal() &&
+                                    (invoke || cacheCtx.loadPreviousValue()) &&
+                                    !txEntry.skipStore();
 
                                 v = cached.innerGet(
                                     null,
                                     this,
                                     /*swap*/true,
                                     readThrough,
-                                    /*failFast*/false,
-                                    /*unmarshal*/true,
                                     /*metrics*/!invoke,
                                     /*event*/!invoke && !dht(),
                                     /*temporary*/false,
@@ -2889,7 +2883,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 IgniteInternalFuture<Boolean> fut = cacheCtx.cache().txLockAsync(enlisted,
                     timeout,
                     this,
-                    false,
+                    /*read*/entryProcessor != null, // Needed to force load from store.
                     retval,
                     isolation,
                     isInvalidate(),
@@ -3069,7 +3063,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 IgniteInternalFuture<Boolean> fut = cacheCtx.cache().txLockAsync(enlisted,
                     timeout,
                     this,
-                    false,
+                    /*read*/invokeMap != null, // Needed to force load from store.
                     retval,
                     isolation,
                     isInvalidate(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 4599060..9dc6a7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -1621,7 +1621,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                         expiryTime,
                         false,
                         topVer,
-                        GridDrType.DR_LOAD);
+                        GridDrType.DR_LOAD,
+                        false);
 
                     cctx.evicts().touch(entry, topVer);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 09cf7c8..ebca870 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -432,8 +432,6 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         @Nullable IgniteInternalTx tx,
         boolean readSwap,
         boolean readThrough,
-        boolean failFast,
-        boolean unmarshal,
         boolean updateMetrics,
         boolean evt,
         boolean tmp,
@@ -668,7 +666,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         long expireTime,
         boolean preload,
         AffinityTopologyVersion topVer,
-        GridDrType drType
+        GridDrType drType,
+        boolean fromStore
     ) throws IgniteCheckedException, GridCacheEntryRemovedException {
         assert false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
index ce60232..45b6e9d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
@@ -54,7 +54,7 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** */
-    protected static final Map<Object, Object> storeMap = new ConcurrentHashMap8<>();
+    public static final Map<Object, Object> storeMap = new ConcurrentHashMap8<>();
 
     /**
      * @return Grids count to start.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughAbstractTest.java
new file mode 100644
index 0000000..294ebea
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughAbstractTest.java
@@ -0,0 +1,382 @@
+/*
+ * 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;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import javax.cache.configuration.Factory;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorResult;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+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.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public abstract class IgniteCacheInvokeReadThroughAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static volatile boolean failed;
+
+    /** */
+    protected boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        failed = false;
+
+        startNodes();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        IgniteCacheAbstractTest.storeMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @return Store factory.
+     */
+    protected Factory<CacheStore> cacheStoreFactory() {
+        return new IgniteCacheAbstractTest.TestStoreFactory();
+    }
+
+    /**
+     * @param data Data.
+     * @param cacheName Cache name.
+     * @throws Exception If failed.
+     */
+    protected void putDataInStore(Map<Object, Object> data, String cacheName) throws Exception {
+        IgniteCacheAbstractTest.storeMap.putAll(data);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected abstract void startNodes() throws Exception;
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    protected void invokeReadThrough(CacheConfiguration ccfg) throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        ignite0.createCache(ccfg);
+
+        try {
+            int key = 0;
+
+            for (Ignite node : G.allGrids()) {
+                if (node.configuration().isClientMode() && ccfg.getNearConfiguration() != null)
+                    node.createNearCache(ccfg.getName(), ccfg.getNearConfiguration());
+            }
+
+            for (Ignite node : G.allGrids()) {
+                log.info("Test for node: " + node.name());
+
+                IgniteCache<Object, Object> cache = node.cache(ccfg.getName());
+
+                for (int i = 0; i < 50; i++)
+                    checkReadThrough(cache, key++, null, null);
+
+                Set<Object> keys = new HashSet<>();
+
+                for (int i = 0; i < 5; i++)
+                    keys.add(key++);
+
+                checkReadThroughInvokeAll(cache, keys, null, null);
+
+                keys = new HashSet<>();
+
+                for (int i = 0; i < 100; i++)
+                    keys.add(key++);
+
+                checkReadThroughInvokeAll(cache, keys, null, null);
+
+                if (ccfg.getAtomicityMode() == TRANSACTIONAL) {
+                    for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+                        for (TransactionIsolation isolation : TransactionIsolation.values()) {
+                            log.info("Test tx [concurrency=" + concurrency + ", isolation=" + isolation + ']');
+
+                            for (int i = 0; i < 50; i++)
+                                checkReadThrough(cache, key++, concurrency, isolation);
+
+                            keys = new HashSet<>();
+
+                            for (int i = 0; i < 5; i++)
+                                keys.add(key++);
+
+                            checkReadThroughInvokeAll(cache, keys, concurrency, isolation);
+
+                            keys = new HashSet<>();
+
+                            for (int i = 0; i < 100; i++)
+                                keys.add(key++);
+
+                            checkReadThroughInvokeAll(cache, keys, concurrency, isolation);
+                        }
+                    }
+
+                    for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+                        for (TransactionIsolation isolation : TransactionIsolation.values()) {
+                            log.info("Test tx2 [concurrency=" + concurrency + ", isolation=" + isolation + ']');
+
+                            for (int i = 0; i < 50; i++)
+                                checkReadThroughGetAndInvoke(cache, key++, concurrency, isolation);
+                        }
+                    }
+                }
+            }
+
+            ignite0.cache(ccfg.getName()).removeAll();
+        }
+        finally {
+            ignite0.destroyCache(ccfg.getName());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @throws Exception If failed.
+     */
+    private void checkReadThrough(IgniteCache<Object, Object> cache,
+        Object key,
+        @Nullable TransactionConcurrency concurrency,
+        @Nullable TransactionIsolation isolation) throws Exception {
+        putDataInStore(Collections.singletonMap(key, key), cache.getName());
+
+        Transaction tx = isolation != null ? cache.unwrap(Ignite.class).transactions().txStart(concurrency, isolation)
+            : null;
+
+        try {
+            Object ret = cache.invoke(key, new TestEntryProcessor());
+
+            assertEquals(key, ret);
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        checkValue(cache.getName(), key, (Integer)key + 1);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @throws Exception If failed.
+     */
+    private void checkReadThroughGetAndInvoke(IgniteCache<Object, Object> cache,
+        Object key,
+        TransactionConcurrency concurrency,
+        TransactionIsolation isolation) throws Exception {
+        putDataInStore(Collections.singletonMap(key, key), cache.getName());
+
+        try (Transaction tx = cache.unwrap(Ignite.class).transactions().txStart(concurrency, isolation)) {
+            cache.get(key);
+
+            Object ret = cache.invoke(key, new TestEntryProcessor());
+
+            assertEquals(key, ret);
+
+            tx.commit();
+        }
+
+        checkValue(cache.getName(), key, (Integer)key + 1);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param keys Key.
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @throws Exception If failed.
+     */
+    private void checkReadThroughInvokeAll(IgniteCache<Object, Object> cache,
+        Set<Object> keys,
+        @Nullable TransactionConcurrency concurrency,
+        @Nullable TransactionIsolation isolation) throws Exception {
+        Map<Object, Object> data = U.newHashMap(keys.size());
+
+        for (Object key : keys)
+            data.put(key, key);
+
+        putDataInStore(data, cache.getName());
+
+        Transaction tx = isolation != null ? cache.unwrap(Ignite.class).transactions().txStart(concurrency, isolation)
+            : null;
+
+        try {
+            Map<Object, EntryProcessorResult<Object>> ret = cache.invokeAll(keys, new TestEntryProcessor());
+
+            assertEquals(ret.size(), keys.size());
+
+            for (Object key : keys) {
+                EntryProcessorResult<Object> res = ret.get(key);
+
+                assertNotNull(res);
+                assertEquals(key, res.get());
+            }
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        for (Object key : keys)
+            checkValue(cache.getName(), key, (Integer)key + 1);
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param key Key.
+     * @param val Expected value.
+     */
+    private void checkValue(String cacheName, Object key, Object val) {
+        for (Ignite ignite : G.allGrids()) {
+            assertEquals("Unexpected value for node: " + ignite.name(),
+                val,
+                ignite.cache(cacheName).get(key));
+        }
+
+        assertFalse(failed);
+    }
+
+    /**
+     * @param cacheMode Cache mode.
+     * @param atomicityMode Atomicity mode.
+     * @param memoryMode Memory mode.
+     * @param backups Number of backups.
+     * @param nearCache Near cache flag.
+     * @return Cache configuration.
+     */
+    @SuppressWarnings("unchecked")
+    protected CacheConfiguration cacheConfiguration(CacheMode cacheMode,
+        CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memoryMode,
+        int backups,
+        boolean nearCache) {
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setReadThrough(true);
+        ccfg.setWriteThrough(true);
+        ccfg.setCacheStoreFactory(cacheStoreFactory());
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg.setAtomicWriteOrderMode(PRIMARY);
+        ccfg.setMemoryMode(memoryMode);
+
+        if (nearCache)
+            ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(backups);
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    static class TestEntryProcessor implements EntryProcessor<Object, Object, Object> {
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... args) {
+            if (!entry.exists()) {
+                failed = true;
+
+                fail();
+            }
+
+            Integer val = (Integer)entry.getValue();
+
+            if (!val.equals(entry.getKey())) {
+                failed = true;
+
+                assertEquals(val, entry.getKey());
+            }
+
+            entry.setValue(val + 1);
+
+            return val;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughSingleNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughSingleNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughSingleNodeTest.java
new file mode 100644
index 0000000..b451abf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughSingleNodeTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class IgniteCacheInvokeReadThroughSingleNodeTest extends IgniteCacheInvokeReadThroughAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void startNodes() throws Exception {
+        startGrid(0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED, 1, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, OFFHEAP_TIERED, 1, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomicNearCache() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED, 1, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomicReplicated() throws Exception {
+        invokeReadThrough(cacheConfiguration(REPLICATED, ATOMIC, ONHEAP_TIERED, 0, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomicLocal() throws Exception {
+        invokeReadThrough(cacheConfiguration(LOCAL, ATOMIC, ONHEAP_TIERED, 0, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED, 1, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, OFFHEAP_TIERED, 1, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxNearCache() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED, 1, true));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxReplicated() throws Exception {
+        invokeReadThrough(cacheConfiguration(REPLICATED, TRANSACTIONAL, ONHEAP_TIERED, 0, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxLocal() throws Exception {
+        invokeReadThrough(cacheConfiguration(LOCAL, TRANSACTIONAL, ONHEAP_TIERED, 0, false));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e10ffef0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
index 2464e81..9578227 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
@@ -17,111 +17,163 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import javax.cache.configuration.Factory;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.MutableEntry;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.store.CacheStore;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
 
 /**
  *
  */
-public class IgniteCacheInvokeReadThroughTest extends IgniteCacheAbstractTest {
+public class IgniteCacheInvokeReadThroughTest extends IgniteCacheInvokeReadThroughAbstractTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-114");
-    }
+    @Override protected void startNodes() throws Exception {
+        startGridsMultiThreaded(4);
 
-    /** */
-    private static volatile boolean failed;
+        client = true;
 
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
+        startGrid(4);
     }
 
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic0() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED, 0, false));
     }
 
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic1() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED, 1, false));
     }
 
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic2() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED, 2, false));
     }
 
-    /** {@inheritDoc} */
-    @Override protected Factory<CacheStore> cacheStoreFactory() {
-        return new TestStoreFactory();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomicNearCache() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, ONHEAP_TIERED, 1, true));
     }
 
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        failed = false;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomicReplicated() throws Exception {
+        invokeReadThrough(cacheConfiguration(REPLICATED, ATOMIC, ONHEAP_TIERED, 0, false));
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testInvokeReadThrough() throws Exception {
-        IgniteCache<Integer, Integer> cache = jcache(0);
-
-        checkReadThrough(cache, primaryKey(cache));
+    public void testInvokeReadThroughAtomic0_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, OFFHEAP_TIERED, 0, false));
+    }
 
-        checkReadThrough(cache, backupKey(cache));
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic1_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, OFFHEAP_TIERED, 1, false));
+    }
 
-        checkReadThrough(cache, nearKey(cache));
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomic2_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, OFFHEAP_TIERED, 2, false));
     }
 
     /**
-     * @param cache Cache.
-     * @param key Key.
+     * @throws Exception If failed.
      */
-    private void checkReadThrough(IgniteCache<Integer, Integer> cache, Integer key) {
-        log.info("Test [key=" + key + ']');
+    public void testInvokeReadThroughAtomicNearCache_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, ATOMIC, OFFHEAP_TIERED, 1, true));
+    }
 
-        storeMap.put(key, key);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughAtomicReplicated_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(REPLICATED, ATOMIC, OFFHEAP_TIERED, 0, false));
+    }
 
-        Object ret = cache.invoke(key, new EntryProcessor<Integer, Integer, Object>() {
-            @Override public Object process(MutableEntry<Integer, Integer> entry, Object... args) {
-                if (!entry.exists()) {
-                    failed = true;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx0() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED, 0, false));
+    }
 
-                    fail();
-                }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx1() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED, 1, false));
+    }
 
-                Integer val = entry.getValue();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx2() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED, 2, false));
+    }
 
-                if (!val.equals(entry.getKey())) {
-                    failed = true;
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxNearCache() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, ONHEAP_TIERED, 1, true));
+    }
 
-                    assertEquals(val, entry.getKey());
-                }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxReplicated() throws Exception {
+        invokeReadThrough(cacheConfiguration(REPLICATED, TRANSACTIONAL, ONHEAP_TIERED, 0, false));
+    }
 
-                entry.setValue(val + 1);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx0_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, OFFHEAP_TIERED, 0, false));
+    }
 
-                return val;
-            }
-        });
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx1_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, OFFHEAP_TIERED, 1, false));
+    }
 
-        assertEquals(key, ret);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTx2_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, OFFHEAP_TIERED, 2, false));
+    }
 
-        for (int i = 0; i < gridCount(); i++)
-            assertEquals("Unexpected value for node: " + i, key + 1, jcache(i).get(key));
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxNearCache_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(PARTITIONED, TRANSACTIONAL, OFFHEAP_TIERED, 1, true));
+    }
 
-        assertFalse(failed);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReadThroughTxReplicated_Offheap() throws Exception {
+        invokeReadThrough(cacheConfiguration(REPLICATED, TRANSACTIONAL, OFFHEAP_TIERED, 0, false));
     }
 }


[22/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh
new file mode 100644
index 0000000..a3a0601
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh
@@ -0,0 +1,384 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+S3_ROOT=s3://bucket/folder
+S3_DOWNLOADS=$S3_ROOT/test
+S3_SYSTEM=$S3_ROOT/test1
+
+IGNITE_DOWNLOAD_URL=$S3_DOWNLOADS/apache-ignite-fabric-1.6.0-SNAPSHOT-bin.zip
+IGNITE_ZIP=apache-ignite-fabric-1.6.0-SNAPSHOT-bin.zip
+IGNITE_UNZIP_DIR=apache-ignite-fabric-1.6.0-SNAPSHOT-bin
+
+TESTS_PACKAGE_DONLOAD_URL=$S3_DOWNLOADS/ignite-cassandra-tests-1.6.0-SNAPSHOT.zip
+TESTS_PACKAGE_ZIP=ignite-cassandra-tests-1.6.0-SNAPSHOT.zip
+TESTS_PACKAGE_UNZIP_DIR=ignite-cassandra-tests
+
+S3_LOGS_URL=$S3_SYSTEM/logs/i-logs
+S3_LOGS_TRIGGER_URL=$S3_SYSTEM/logs-trigger
+S3_BOOTSTRAP_SUCCESS_URL=$S3_SYSTEM/i-success
+S3_BOOTSTRAP_FAILURE_URL=$S3_SYSTEM/i-failure
+S3_CASSANDRA_NODES_DISCOVERY_URL=$S3_SYSTEM/c-discovery
+S3_IGNITE_NODES_DISCOVERY_URL=$S3_SYSTEM/i-discovery
+S3_IGNITE_FIRST_NODE_LOCK_URL=$S3_SYSTEM/i-first-node-lock
+S3_IGNITE_NODES_JOIN_LOCK_URL=$S3_SYSTEM/i-join-lock
+
+INSTANCE_REGION=us-west-2
+INSTANCE_NAME_TAG=IGNITE-SERVER
+INSTANCE_OWNER_TAG=ignite@apache.org
+INSTANCE_PROJECT_TAG=ignite
+
+terminate()
+{
+    if [[ "$S3_BOOTSTRAP_SUCCESS_URL" != */ ]]; then
+        S3_BOOTSTRAP_SUCCESS_URL=${S3_BOOTSTRAP_SUCCESS_URL}/
+    fi
+
+    if [[ "$S3_BOOTSTRAP_FAILURE_URL" != */ ]]; then
+        S3_BOOTSTRAP_FAILURE_URL=${S3_BOOTSTRAP_FAILURE_URL}/
+    fi
+
+    host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+    msg=$host_name
+
+    if [ -n "$1" ]; then
+        echo "[ERROR] $1"
+        echo "[ERROR]-----------------------------------------------------"
+        echo "[ERROR] Ignite node bootstrap failed"
+        echo "[ERROR]-----------------------------------------------------"
+        msg=$1
+        reportFolder=${S3_BOOTSTRAP_FAILURE_URL}${host_name}
+        reportFile=$reportFolder/__error__
+    else
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Ignite node bootstrap successfully completed"
+        echo "[INFO]-----------------------------------------------------"
+        reportFolder=${S3_BOOTSTRAP_SUCCESS_URL}${host_name}
+        reportFile=$reportFolder/__success__
+    fi
+
+    echo $msg > /opt/bootstrap-result
+
+    aws s3 rm --recursive $reportFolder
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed drop report folder: $reportFolder"
+    fi
+
+    aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+    fi
+
+    rm -f /opt/bootstrap-result
+
+    if [ -n "$1" ]; then
+        exit 1
+    fi
+
+    exit 0
+}
+
+tagInstance()
+{
+    export EC2_HOME=/opt/aws/apitools/ec2
+    export JAVA_HOME=/opt/jdk1.8.0_77
+    export PATH=$JAVA_HOME/bin:$EC2_HOME/bin:$PATH
+
+    INSTANCE_ID=$(curl http://169.254.169.254/latest/meta-data/instance-id)
+    if [ $? -ne 0 ]; then
+        terminate "Failed to get instance metadata to tag it"
+    fi
+
+    if [ -n "$INSTANCE_NAME_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag Name=${INSTANCE_NAME_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: Name=${INSTANCE_NAME_TAG}"
+        fi
+    fi
+
+    if [ -n "$INSTANCE_OWNER_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag owner=${INSTANCE_OWNER_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: owner=${INSTANCE_OWNER_TAG}"
+        fi
+    fi
+
+    if [ -n "$INSTANCE_PROJECT_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag project=${INSTANCE_PROJECT_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: project=${INSTANCE_PROJECT_TAG}"
+        fi
+    fi
+}
+
+downloadPackage()
+{
+    echo "[INFO] Downloading $3 package from $1 into $2"
+
+    if [[ "$1" == s3* ]]; then
+        aws s3 cp $1 $2
+
+        if [ $? -ne 0 ]; then
+            echo "[WARN] Failed to download $3 package from first attempt"
+            rm -Rf $2
+            sleep 10s
+
+            echo "[INFO] Trying second attempt to download $3 package"
+            aws s3 cp $1 $2
+
+            if [ $? -ne 0 ]; then
+                echo "[WARN] Failed to download $3 package from second attempt"
+                rm -Rf $2
+                sleep 10s
+
+                echo "[INFO] Trying third attempt to download $3 package"
+                aws s3 cp $1 $2
+
+                if [ $? -ne 0 ]; then
+                    terminate "All three attempts to download $3 package from $1 are failed"
+                fi
+            fi
+        fi
+    else
+        curl "$1" -o "$2"
+
+        if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+            echo "[WARN] Failed to download $3 package from first attempt"
+            rm -Rf $2
+            sleep 10s
+
+            echo "[INFO] Trying second attempt to download $3 package"
+            curl "$1" -o "$2"
+
+            if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+                echo "[WARN] Failed to download $3 package from second attempt"
+                rm -Rf $2
+                sleep 10s
+
+                echo "[INFO] Trying third attempt to download $3 package"
+                curl "$1" -o "$2"
+
+                if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+                    terminate "All three attempts to download $3 package from $1 are failed"
+                fi
+            fi
+        fi
+    fi
+
+    echo "[INFO] $3 package successfully downloaded from $1 into $2"
+}
+
+if [[ "$S3_IGNITE_NODES_DISCOVERY_URL" != */ ]]; then
+    S3_IGNITE_NODES_DISCOVERY_URL=${S3_IGNITE_NODES_DISCOVERY_URL}/
+fi
+
+if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+    S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Ignite node"
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Ignite download URL: $IGNITE_DOWNLOAD_URL"
+echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+echo "[INFO] Logs URL: $S3_LOGS_URL"
+echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER_URL"
+echo "[INFO] Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY_URL"
+echo "[INFO] Ignite first node lock URL: $S3_IGNITE_FIRST_NODE_LOCK_URL"
+echo "[INFO] Ignite nodes join lock URL: $S3_IGNITE_NODES_JOIN_LOCK_URL"
+echo "[INFO] Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+echo "[INFO] Bootsrap success URL: $S3_BOOTSTRAP_SUCCESS_URL"
+echo "[INFO] Bootsrap failure URL: $S3_BOOTSTRAP_FAILURE_URL"
+echo "[INFO]-----------------------------------------------------------------"
+
+echo "[INFO] Installing 'wget' package"
+yum -y install wget
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'wget' package"
+fi
+
+echo "[INFO] Installing 'net-tools' package"
+yum -y install net-tools
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'net-tools' package"
+fi
+
+echo "[INFO] Installing 'python' package"
+yum -y install python
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'python' package"
+fi
+
+echo "[INFO] Installing 'unzip' package"
+yum -y install unzip
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'unzip' package"
+fi
+
+rm -Rf /opt/jdk1.8.0_77 /opt/jdk-8u77-linux-x64.tar.gz
+
+echo "[INFO] Downloading 'jdk-8u77'"
+wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz" -O /opt/jdk-8u77-linux-x64.tar.gz
+if [ $? -ne 0 ]; then
+    terminate "Failed to download 'jdk-8u77'"
+fi
+
+echo "[INFO] Unzipping 'jdk-8u77'"
+tar -xvzf /opt/jdk-8u77-linux-x64.tar.gz -C /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to untar 'jdk-8u77'"
+fi
+
+rm -Rf /opt/jdk-8u77-linux-x64.tar.gz
+
+downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+echo "[INFO] Installing 'pip'"
+python /opt/get-pip.py
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'pip'"
+fi
+
+echo "[INFO] Installing 'awscli'"
+pip install --upgrade awscli
+if [ $? -ne 0 ]; then
+    echo "[ERROR] Failed to install 'awscli' using pip"
+    echo "[INFO] Trying to install awscli using zip archive"
+    echo "[INFO] Downloading awscli zip"
+
+    downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+    echo "[INFO] Unzipping awscli zip"
+    unzip /opt/awscli-bundle.zip -d /opt
+    if [ $? -ne 0 ]; then
+        terminate "Failed to unzip awscli zip"
+    fi
+
+    rm -fR /opt/awscli-bundle.zip
+
+    echo "[INFO] Installing awscli"
+    /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+    if [ $? -ne 0 ]; then
+        terminate "Failed to install awscli"
+    fi
+
+    echo "[INFO] Successfully installed awscli from zip archive"
+fi
+
+tagInstance
+
+echo "[INFO] Creating 'ignite' group"
+exists=$(cat /etc/group | grep ignite)
+if [ -z "$exists" ]; then
+    groupadd ignite
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create 'ignite' group"
+    fi
+fi
+
+echo "[INFO] Creating 'ignite' user"
+exists=$(cat /etc/passwd | grep ignite)
+if [ -z "$exists" ]; then
+    useradd -g ignite ignite
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create 'ignite' user"
+    fi
+fi
+
+rm -Rf /opt/ignite /opt/$IGNITE_ZIP
+
+downloadPackage "$IGNITE_DOWNLOAD_URL" "/opt/$IGNITE_ZIP" "Ignite"
+
+echo "[INFO] Unzipping Ignite package"
+unzip /opt/$IGNITE_ZIP -d /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to unzip Ignite package"
+fi
+
+rm -Rf /opt/$IGNITE_ZIP /opt/ignite-start.sh /opt/ignite-env.sh /opt/ignite
+mv /opt/$IGNITE_UNZIP_DIR /opt/ignite
+chown -R ignite:ignite /opt/ignite
+
+downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/$TESTS_PACKAGE_ZIP" "Tests"
+
+unzip /opt/$TESTS_PACKAGE_ZIP -d /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to unzip tests package: $TESTS_PACKAGE_DONLOAD_URL"
+fi
+
+chown -R ignite:ignite /opt/$TESTS_PACKAGE_UNZIP_DIR
+find /opt/$TESTS_PACKAGE_UNZIP_DIR -type f -name "*.sh" -exec chmod ug+x {} \;
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/ignite/ignite-start.sh" ]; then
+    terminate "There are no ignite-start.sh in tests package"
+fi
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/ignite/ignite-cassandra-server-template.xml" ]; then
+    terminate "There are no ignite-cassandra-server-template.xml in tests package"
+fi
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/logs-collector.sh" ]; then
+    terminate "There are no logs-collector.sh in tests package"
+fi
+
+testsJar=$(find /opt/$TESTS_PACKAGE_UNZIP_DIR -type f -name "*.jar" | grep ignite-cassandra- | grep tests.jar)
+if [ -n "$testsJar" ]; then
+    echo "[INFO] Coping tests jar $testsJar into /opt/ignite/libs/optional/ignite-cassandra"
+    cp $testsJar /opt/ignite/libs/optional/ignite-cassandra
+    if [ $? -ne 0 ]; then
+        terminate "Failed copy $testsJar into /opt/ignite/libs/optional/ignite-cassandra"
+    fi
+fi
+
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/ignite/ignite-start.sh /opt
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/ignite/ignite-cassandra-server-template.xml /opt/ignite/config
+mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/logs-collector.sh /opt
+
+if [ -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/ignite/ignite-env.sh" ]; then
+    mv -f /opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/ignite/ignite-env.sh /opt
+    chown -R ignite:ignite /opt/ignite-env.sh
+fi
+
+rm -Rf /opt/$TESTS_PACKAGE_UNZIP_DIR
+chown -R ignite:ignite /opt/ignite-start.sh /opt/logs-collector.sh /opt/ignite/config/ignite-cassandra-server-template.xml
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+echo "export JAVA_HOME=/opt/jdk1.8.0_77" >> $profile
+echo "export IGNITE_HOME=/opt/ignite" >> $profile
+echo "export USER_LIBS=\$IGNITE_HOME/libs/optional/ignite-cassandra/*:\$IGNITE_HOME/libs/optional/ignite-slf4j/*" >> $profile
+echo "export PATH=\$JAVA_HOME/bin:\IGNITE_HOME/bin:\$PATH" >> $profile
+echo "export S3_BOOTSTRAP_SUCCESS_URL=$S3_BOOTSTRAP_SUCCESS_URL" >> $profile
+echo "export S3_BOOTSTRAP_FAILURE_URL=$S3_BOOTSTRAP_FAILURE_URL" >> $profile
+echo "export S3_CASSANDRA_NODES_DISCOVERY_URL=$S3_CASSANDRA_NODES_DISCOVERY_URL" >> $profile
+echo "export S3_IGNITE_NODES_DISCOVERY_URL=$S3_IGNITE_NODES_DISCOVERY_URL" >> $profile
+echo "export S3_IGNITE_NODES_JOIN_LOCK_URL=$S3_IGNITE_NODES_JOIN_LOCK_URL" >> $profile
+echo "export S3_IGNITE_FIRST_NODE_LOCK_URL=$S3_IGNITE_FIRST_NODE_LOCK_URL" >> $profile
+
+HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+/opt/logs-collector.sh "/opt/ignite/work/log" "$S3_LOGS_URL/$HOST_NAME" "$S3_LOGS_TRIGGER_URL" > /opt/ignite/logs-collector.log &
+
+cmd="/opt/ignite-start.sh"
+
+#sudo -u ignite -g ignite sh -c "$cmd | tee /opt/ignite/start.log"
+
+$cmd | tee /opt/ignite/start.log
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml
new file mode 100644
index 0000000..f85dcd9
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml
@@ -0,0 +1,177 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <!-- Cassandra connection settings -->
+    <bean id="loadBalancingPolicy" class="com.datastax.driver.core.policies.RoundRobinPolicy"/>
+
+    <util:list id="contactPoints" value-type="java.lang.String">
+        ${CASSANDRA_SEEDS}
+    </util:list>
+
+    <bean id="cassandraAdminDataSource" class="org.apache.ignite.cache.store.cassandra.datasource.DataSource">
+        <property name="user" value="cassandra"/>
+        <property name="password" value="cassandra"/>
+        <property name="contactPoints" ref="contactPoints"/>
+        <property name="readConsistency" value="ONE"/>
+        <property name="writeConsistency" value="ONE"/>
+        <property name="loadBalancingPolicy" ref="loadBalancingPolicy"/>
+    </bean>
+
+    <!-- Persistence settings for 'cache1' -->
+    <bean id="cache1_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="primitive_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" column="key"/>
+    <valuePersistence class="java.lang.Long" strategy="PRIMITIVE" column="value"/>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Persistence settings for 'cache2' -->
+    <bean id="cache2_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="blob_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" />
+    <valuePersistence strategy="BLOB"/>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Persistence settings for 'cache3' -->
+    <bean id="cache3_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="pojo_test3" ttl="86400">
+    <keyspaceOptions>
+        REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 3}
+        AND DURABLE_WRITES = true
+    </keyspaceOptions>
+    <tableOptions>
+        comment = 'A most excellent and useful table'
+        AND read_repair_chance = 0.2
+    </tableOptions>
+    <keyPersistence class="org.apache.ignite.tests.pojos.PersonId" strategy="POJO">
+        <partitionKey>
+            <field name="companyCode" column="company" />
+            <field name="departmentCode" column="department" />
+        </partitionKey>
+        <clusterKey>
+            <field name="personNumber" column="number" sort="desc"/>
+        </clusterKey>
+    </keyPersistence>
+    <valuePersistence class="org.apache.ignite.tests.pojos.Person"
+                      strategy="POJO"
+                      serializer="org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer">
+        <field name="firstName" column="first_name" />
+        <field name="lastName" column="last_name" />
+        <field name="age" />
+        <field name="married" index="true"/>
+        <field name="height" />
+        <field name="weight" />
+        <field name="birthDate" column="birth_date" />
+        <field name="phones" />
+    </valuePersistence>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Partitioned cache example configuration (Atomic mode). -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                </bean>
+
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="writeBehindEnabled" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache2" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="writeBehindEnabled" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache2_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache3" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache3"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="writeBehindEnabled" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache3_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                ${IGNITE_SEEDS}
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-env.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-env.sh b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-env.sh
new file mode 100644
index 0000000..e0643b7
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-env.sh
@@ -0,0 +1,25 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+JVM_OPTS="-Xms10g -Xmx10g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m"
+JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+UseTLAB -XX:NewSize=128m -XX:MaxNewSize=768m"
+#JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=0 -XX:SurvivorRatio=1024 -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=60"
+JVM_OPTS="$JVM_OPTS -Xss16m"
+
+export JVM_OPTS

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-start.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-start.sh b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-start.sh
new file mode 100644
index 0000000..bb1ff0c
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/ignite/ignite-start.sh
@@ -0,0 +1,637 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+
+terminate()
+{
+    if [[ "$S3_BOOTSTRAP_SUCCESS_URL" != */ ]]; then
+        S3_BOOTSTRAP_SUCCESS_URL=${S3_BOOTSTRAP_SUCCESS_URL}/
+    fi
+
+    if [[ "$S3_BOOTSTRAP_FAILURE_URL" != */ ]]; then
+        S3_BOOTSTRAP_FAILURE_URL=${S3_BOOTSTRAP_FAILURE_URL}/
+    fi
+
+    msg=$HOST_NAME
+
+    if [ -n "$1" ]; then
+        echo "[ERROR] $1"
+        echo "[ERROR]-----------------------------------------------------"
+        echo "[ERROR] Failed to start Ignite node"
+        echo "[ERROR]-----------------------------------------------------"
+        msg=$1
+        reportFolder=${S3_BOOTSTRAP_FAILURE_URL}${HOST_NAME}
+        reportFile=$reportFolder/__error__
+    else
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Ignite node successfully started"
+        echo "[INFO]-----------------------------------------------------"
+        reportFolder=${S3_BOOTSTRAP_SUCCESS_URL}${HOST_NAME}
+        reportFile=$reportFolder/__success__
+    fi
+
+    echo $msg > /opt/ignite/start_result
+
+    aws s3 rm --recursive $reportFolder
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed drop report folder: $reportFolder"
+    fi
+
+    if [ -d "/opt/ignite/work/log" ]; then
+        aws s3 sync --sse AES256 /opt/ignite/work/log $reportFolder
+        if [ $? -ne 0 ]; then
+            echo "[ERROR] Failed to export Ignite logs to: $reportFolder"
+        fi
+    fi
+
+    aws s3 cp --sse AES256 /opt/ignite/start_result $reportFile
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to export node start result to: $reportFile"
+    fi
+
+    rm -f /opt/ignite/start_result /opt/ignite/join-lock /opt/ignite/remote-join-lock
+
+    if [ -n "$1" ]; then
+        exit 1
+    fi
+
+    exit 0
+}
+
+registerNode()
+{
+    echo "[INFO] Registering Ignite node seed: ${S3_IGNITE_NODES_DISCOVERY_URL}$HOST_NAME"
+
+    aws s3 cp --sse AES256 /opt/ignite/join-lock ${S3_IGNITE_NODES_DISCOVERY_URL}$HOST_NAME
+    if [ $? -ne 0 ]; then
+        terminate "Failed to register Ignite node seed: ${S3_IGNITE_NODES_DISCOVERY_URL}$HOST_NAME"
+    fi
+
+    echo "[INFO] Ignite node seed successfully registered"
+}
+
+unregisterNode()
+{
+    echo "[INFO] Removing Ignite node registration from: ${S3_IGNITE_NODES_DISCOVERY_URL}$HOST_NAME"
+    aws s3 rm ${S3_IGNITE_NODES_DISCOVERY_URL}$HOST_NAME
+    echo "[INFO] Ignite node registration removed"
+}
+
+cleanupMetadata()
+{
+    echo "[INFO] Running cleanup"
+    aws s3 rm $S3_IGNITE_NODES_JOIN_LOCK_URL
+    aws s3 rm --recursive $S3_IGNITE_NODES_DISCOVERY_URL
+    aws s3 rm --recursive $S3_BOOTSTRAP_SUCCESS_URL
+    aws s3 rm --recursive $S3_BOOTSTRAP_FAILURE_URL
+    echo "[INFO] Cleanup completed"
+}
+
+setupCassandraSeeds()
+{
+    echo "[INFO] Setting up Cassandra seeds"
+
+    echo "[INFO] Looking for Cassandra seeds in: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+
+    startTime=$(date +%s)
+
+    while true; do
+        seeds=$(aws s3 ls $S3_CASSANDRA_NODES_DISCOVERY_URL | grep -v PRE | sed -r "s/^.* //g")
+        if [ -n "$seeds" ]; then
+            seeds=($seeds)
+            length=${#seeds[@]}
+
+            if [ $length -lt 4 ]; then
+                seed1=${seeds[0]}
+                seed2=${seeds[1]}
+                seed3=${seeds[2]}
+            else
+                pos1=$(($RANDOM%$length))
+                pos2=$(($RANDOM%$length))
+                pos3=$(($RANDOM%$length))
+                seed1=${seeds[${pos1}]}
+                seed2=${seeds[${pos2}]}
+                seed3=${seeds[${pos3}]}
+            fi
+
+            CASSANDRA_SEEDS="<value>$seed1<\/value>"
+
+            if [ "$seed2" != "$seed1" ] && [ -n "$seed2" ]; then
+                CASSANDRA_SEEDS="$CASSANDRA_SEEDS<value>$seed2<\/value>"
+            fi
+
+            if [ "$seed3" != "$seed2" ] && [ "$seed3" != "$seed1" ] && [ -n "$seed3" ]; then
+                CASSANDRA_SEEDS="$CASSANDRA_SEEDS<value>$seed3<\/value>"
+            fi
+
+            echo "[INFO] Using Cassandra seeds: $CASSANDRA_SEEDS"
+
+            cat /opt/ignite/config/ignite-cassandra-server-template.xml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CASSANDRA_SEEDS/g" > /opt/ignite/config/ignite-cassandra-server.xml
+
+            return 0
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but no Cassandra nodes is up and running"
+        fi
+
+        echo "[INFO] Waiting for the first Cassandra node to start and publish its seed, time passed ${duration}min"
+
+        sleep 1m
+    done
+}
+
+setupIgniteSeeds()
+{
+    echo "[INFO] Setting up Ignite seeds"
+
+    if [ "$FIRST_NODE" == "true" ]; then
+        IGNITE_SEEDS="<value>127.0.0.1:47500..47509<\/value>"
+        echo "[INFO] Using localhost address as a seed for the first Ignite node: $IGNITE_SEEDS"
+        aws s3 rm --recursive ${S3_IGNITE_NODES_DISCOVERY_URL::-1}
+        if [ $? -ne 0 ]; then
+            terminate "Failed to clean Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY_URL"
+        fi
+
+        cat /opt/ignite/config/ignite-cassandra-server.xml | sed -r "s/\\\$\{IGNITE_SEEDS\}/$IGNITE_SEEDS/g" > /opt/ignite/config/ignite-cassandra-server1.xml
+        mv -f /opt/ignite/config/ignite-cassandra-server1.xml /opt/ignite/config/ignite-cassandra-server.xml
+
+        return 0
+    fi
+
+    echo "[INFO] Looking for Ignite seeds in: $S3_IGNITE_NODES_DISCOVERY_URL"
+
+    startTime=$(date +%s)
+
+    while true; do
+        seeds=$(aws s3 ls $S3_IGNITE_NODES_DISCOVERY_URL | grep -v PRE | sed -r "s/^.* //g")
+        if [ -n "$seeds" ]; then
+            seeds=($seeds)
+            length=${#seeds[@]}
+
+            if [ $length -lt 4 ]; then
+                seed1=${seeds[0]}
+                seed2=${seeds[1]}
+                seed3=${seeds[2]}
+            else
+                pos1=$(($RANDOM%$length))
+                pos2=$(($RANDOM%$length))
+                pos3=$(($RANDOM%$length))
+                seed1=${seeds[${pos1}]}
+                seed2=${seeds[${pos2}]}
+                seed3=${seeds[${pos3}]}
+            fi
+
+            IGNITE_SEEDS="<value>$seed1<\/value>"
+
+            if [ "$seed2" != "$seed1" ] && [ -n "$seed2" ]; then
+                IGNITE_SEEDS="$IGNITE_SEEDS<value>$seed2<\/value>"
+            fi
+
+            if [ "$seed3" != "$seed2" ] && [ "$seed3" != "$seed1" ] && [ -n "$seed3" ]; then
+                IGNITE_SEEDS="$IGNITE_SEEDS<value>$seed3<\/value>"
+            fi
+
+            echo "[INFO] Using Ignite seeds: $IGNITE_SEEDS"
+
+            cat /opt/ignite/config/ignite-cassandra-server.xml | sed -r "s/\\\$\{IGNITE_SEEDS\}/$IGNITE_SEEDS/g" > /opt/ignite/config/ignite-cassandra-server1.xml
+            mv -f /opt/ignite/config/ignite-cassandra-server1.xml /opt/ignite/config/ignite-cassandra-server.xml
+
+            return 0
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but no Ignite nodes is up and running"
+        fi
+
+        echo "[INFO] Waiting for the first Ignite node to start and publish its seed, time passed ${duration}min"
+
+        sleep 1m
+    done
+}
+
+tryToGetFirstNodeLock()
+{
+    echo "[INFO] Trying to get first node lock"
+
+    checkFirstNodeLockExist
+    if [ $? -ne 0 ]; then
+        return 1
+    fi
+
+    createFirstNodeLock
+
+    sleep 5s
+
+    rm -Rf /opt/ignite/first-node-lock
+
+    aws s3 cp $S3_IGNITE_FIRST_NODE_LOCK_URL /opt/ignite/first-node-lock
+    if [ $? -ne 0 ]; then
+        echo "[WARN] Failed to check just created first node lock"
+        return 1
+    fi
+
+    first_host=$(cat /opt/ignite/first-node-lock)
+
+    rm -f /opt/ignite/first-node-lock
+
+    if [ "$first_host" != "$HOST_NAME" ]; then
+        echo "[INFO] Node $first_host has discarded previously created first node lock"
+        return 1
+    fi
+
+    echo "[INFO] Congratulations, got first node lock"
+
+    return 0
+}
+
+checkFirstNodeLockExist()
+{
+    echo "[INFO] Checking for the first node lock"
+
+    lockExists=$(aws s3 ls $S3_IGNITE_FIRST_NODE_LOCK_URL)
+    if [ -n "$lockExists" ]; then
+        echo "[INFO] First node lock already exists"
+        return 1
+    fi
+
+    echo "[INFO] First node lock doesn't exist yet"
+
+    return 0
+}
+
+createFirstNodeLock()
+{
+    aws s3 cp --sse AES256 /opt/ignite/join-lock $S3_IGNITE_FIRST_NODE_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create first node lock"
+    fi
+    echo "[INFO] Created first node lock"
+}
+
+removeFirstNodeLock()
+{
+    aws s3 rm $S3_IGNITE_FIRST_NODE_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to remove first node lock"
+    fi
+    echo "[INFO] Removed first node lock"
+}
+
+tryToGetClusterJoinLock()
+{
+    echo "[INFO] Trying to get cluster join lock"
+
+    checkClusterJoinLockExist
+    if [ $? -ne 0 ]; then
+        return 1
+    fi
+
+    createClusterJoinLock
+
+    sleep 5s
+
+    rm -Rf /opt/ignite/remote-join-lock
+
+    aws s3 cp $S3_IGNITE_NODES_JOIN_LOCK_URL /opt/ignite/remote-join-lock
+    if [ $? -ne 0 ]; then
+        echo "[WARN] Failed to check just created cluster join lock"
+        return 1
+    fi
+
+    join_host=$(cat /opt/ignite/remote-join-lock)
+
+    if [ "$join_host" != "$HOST_NAME" ]; then
+        echo "[INFO] Node $first_host has discarded previously created cluster join lock"
+        return 1
+    fi
+
+    echo "[INFO] Congratulations, got cluster join lock"
+
+    return 0
+}
+
+checkClusterJoinLockExist()
+{
+    echo "[INFO] Checking for the cluster join lock"
+
+    lockExists=$(aws s3 ls $S3_IGNITE_NODES_JOIN_LOCK_URL)
+    if [ -n "$lockExists" ]; then
+        echo "[INFO] Cluster join lock already exists"
+        return 1
+    fi
+
+    echo "[INFO] Cluster join lock doesn't exist"
+
+    return 0
+}
+
+createClusterJoinLock()
+{
+    aws s3 cp --sse AES256 /opt/ignite/join-lock $S3_IGNITE_NODES_JOIN_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create cluster join lock"
+    fi
+    echo "[INFO] Created cluster join lock"
+}
+
+removeClusterJoinLock()
+{
+    aws s3 rm $S3_IGNITE_NODES_JOIN_LOCK_URL
+    if [ $? -ne 0 ]; then
+        terminate "Failed to remove cluster join lock"
+    fi
+    echo "[INFO] Removed cluster join lock"
+}
+
+waitToJoinIgniteCluster()
+{
+    echo "[INFO] Waiting to join Ignite cluster"
+
+    while true; do
+        tryToGetClusterJoinLock
+
+        if [ $? -ne 0 ]; then
+            echo "[INFO] Another node is trying to join cluster. Waiting for extra 1min."
+            sleep 1m
+        else
+            echo "[INFO]-------------------------------------------------------------"
+            echo "[INFO] Congratulations, got lock to join Ignite cluster"
+            echo "[INFO]-------------------------------------------------------------"
+            break
+        fi
+    done
+}
+
+checkIgniteStatus()
+{
+    proc=$(ps -ef | grep java | grep "org.apache.ignite.startup.cmdline.CommandLineStartup")
+
+    nodeId=
+    nodeAddrs=
+    nodePorts=
+    topology=
+    metrics=
+
+    logFile=$(ls /opt/ignite/work/log/ | grep "\.log$")
+    if [ -n "$logFile" ]; then
+        logFile=/opt/ignite/work/log/$logFile
+        nodeId=$(cat $logFile | grep "Local node \[ID")
+        nodeAddrs=$(cat $logFile | grep "Local node addresses:")
+        nodePorts=$(cat $logFile | grep "Local ports:")
+        topology=$(cat $logFile | grep "Topology snapshot")
+        metrics=$(cat $logFile | grep "Metrics for local node" | head -n 1)
+    fi
+
+    if [ -n "$nodeId" ] && [ -n "$nodeAddrs" ] && [ -n "$nodePorts" ] && [ -n "$topology" ] && [ -n "$metrics" ] && [ -n "$proc" ]; then
+        sleep 30s
+        return 0
+    fi
+
+    return 1
+}
+
+waitFirstIgniteNodeRegistered()
+{
+    echo "[INFO] Waiting for the first Ignite node to register"
+
+    startTime=$(date +%s)
+
+    while true; do
+        first_host=
+
+        exists=$(aws s3 ls $S3_IGNITE_FIRST_NODE_LOCK_URL)
+        if [ -n "$exists" ]; then
+            rm -Rf /opt/ignite/first-node-lock
+
+            aws s3 cp $S3_IGNITE_FIRST_NODE_LOCK_URL /opt/ignite/first-node-lock
+            if [ $? -ne 0 ]; then
+                terminate "Failed to check existing first node lock"
+            fi
+
+            first_host=$(cat /opt/ignite/first-node-lock)
+
+            rm -Rf /opt/ignite/first-node-lock
+        fi
+
+        if [ -n "$first_host" ]; then
+            exists=$(aws s3 ls ${S3_IGNITE_NODES_DISCOVERY_URL}${first_host})
+            if [ -n "$exists" ]; then
+                break
+            fi
+        fi
+
+        currentTime=$(date +%s)
+        duration=$(( $currentTime-$startTime ))
+        duration=$(( $duration/60 ))
+
+        if [ $duration -gt $NODE_STARTUP_TIME ]; then
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but first Ignite node is still not up and running"
+        fi
+
+        echo "[INFO] Waiting extra 1min"
+
+        sleep 1m
+    done
+
+    echo "[INFO] First Ignite node registered"
+}
+
+startIgnite()
+{
+    echo "[INFO]-------------------------------------------------------------"
+    echo "[INFO] Trying attempt $START_ATTEMPT to start Ignite daemon"
+    echo "[INFO]-------------------------------------------------------------"
+    echo ""
+
+    setupCassandraSeeds
+    setupIgniteSeeds
+
+    if [ "$FIRST_NODE" == "true" ]; then
+        aws s3 rm --recursive ${S3_IGNITE_NODES_DISCOVERY_URL::-1}
+        if [ $? -ne 0 ]; then
+            terminate "Failed to clean Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY_URL"
+        fi
+    else
+        waitToJoinIgniteCluster
+    fi
+
+    proc=$(ps -ef | grep java | grep "org.apache.ignite.startup.cmdline.CommandLineStartup")
+    proc=($proc)
+
+    if [ -n "${proc[1]}" ]; then
+        echo "[INFO] Terminating existing Ignite process ${proc[1]}"
+        kill -9 ${proc[1]}
+    fi
+
+    echo "[INFO] Starting Ignite"
+    rm -Rf /opt/ignite/work/*
+    /opt/ignite/bin/ignite.sh /opt/ignite/config/ignite-cassandra-server.xml &
+
+    echo "[INFO] Ignite job id: $!"
+
+    sleep 1m
+
+    START_ATTEMPT=$(( $START_ATTEMPT+1 ))
+}
+
+# Time (in minutes) to wait for Ignite/Cassandra daemon up and running
+NODE_STARTUP_TIME=10
+
+# Number of attempts to start (not first) Ignite daemon
+NODE_START_ATTEMPTS=3
+
+HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+echo $HOST_NAME > /opt/ignite/join-lock
+
+START_ATTEMPT=0
+
+FIRST_NODE="false"
+
+unregisterNode
+
+tryToGetFirstNodeLock
+
+if [ $? -eq 0 ]; then
+    FIRST_NODE="true"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE" == "true" ]; then
+    echo "[INFO] Starting first Ignite node"
+else
+    echo "[INFO] Starting Ignite node"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Ignite nodes discovery URL: $S3_IGNITE_NODES_DISCOVERY_URL"
+echo "[INFO] Ignite first node lock URL: $S3_IGNITE_FIRST_NODE_LOCK_URL"
+echo "[INFO] Cassandra nodes discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+echo "[INFO] Start success URL: $S3_BOOTSTRAP_SUCCESS_URL"
+echo "[INFO] Start failure URL: $S3_BOOTSTRAP_FAILURE_URL"
+echo "[INFO] IGNITE_HOME: $IGNITE_HOME"
+echo "[INFO] JAVA_HOME: $JAVA_HOME"
+echo "[INFO] PATH: $PATH"
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ -z "$S3_CASSANDRA_NODES_DISCOVERY_URL" ]; then
+    terminate "Cassandra S3 discovery URL doesn't specified"
+fi
+
+if [[ "$S3_CASSANDRA_NODES_DISCOVERY_URL" != */ ]]; then
+    S3_CASSANDRA_NODES_DISCOVERY_URL=${S3_CASSANDRA_NODES_DISCOVERY_URL}/
+fi
+
+if [ -z "$S3_IGNITE_NODES_DISCOVERY_URL" ]; then
+    terminate "Ignite S3 discovery URL doesn't specified"
+fi
+
+if [[ "$S3_IGNITE_NODES_DISCOVERY_URL" != */ ]]; then
+    S3_IGNITE_NODES_DISCOVERY_URL=${S3_IGNITE_NODES_DISCOVERY_URL}/
+fi
+
+if [ "$FIRST_NODE" != "true" ]; then
+    waitFirstIgniteNodeRegistered
+else
+    cleanupMetadata
+fi
+
+envScript=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/ignite-env.sh)
+if [ -f "$envScript" ]; then
+    . $envScript
+fi
+
+startIgnite
+
+startTime=$(date +%s)
+
+while true; do
+    proc=$(ps -ef | grep java | grep "org.apache.ignite.startup.cmdline.CommandLineStartup")
+
+    checkIgniteStatus
+
+    if [ $? -eq 0 ]; then
+        sleep 1m
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Ignite daemon successfully started"
+        echo "[INFO]-----------------------------------------------------"
+        echo $proc
+        echo "[INFO]-----------------------------------------------------"
+
+        if [ "$FIRST_NODE" != "true" ]; then
+            removeClusterJoinLock
+        fi
+
+        break
+    fi
+
+    currentTime=$(date +%s)
+    duration=$(( $currentTime-$startTime ))
+    duration=$(( $duration/60 ))
+
+    if [ $duration -gt $NODE_STARTUP_TIME ]; then
+        if [ "$FIRST_NODE" == "true" ]; then
+            removeFirstNodeLock
+            terminate "${NODE_STARTUP_TIME}min timeout expired, but first Ignite daemon is still not up and running"
+        else
+            removeClusterJoinLock
+
+            if [ $START_ATTEMPT -gt $NODE_START_ATTEMPTS ]; then
+                terminate "${NODE_START_ATTEMPTS} attempts exceed, but Ignite daemon is still not up and running"
+            fi
+
+            startIgnite
+        fi
+
+        continue
+    fi
+
+    if [ -z "$proc" ]; then
+        if [ "$FIRST_NODE" == "true" ]; then
+            removeFirstNodeLock
+            terminate "Failed to start Ignite daemon"
+        fi
+
+        removeClusterJoinLock
+        echo "[WARN] Failed to start Ignite daemon. Sleeping for extra 1min"
+        sleep 1m
+        startIgnite
+        continue
+    fi
+
+    echo "[INFO] Waiting for Ignite daemon to start, time passed ${duration}min"
+    sleep 30s
+done
+
+registerNode
+
+terminate
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/logs-collector.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/logs-collector.sh b/modules/cassandra/src/test/bootstrap/aws/logs-collector.sh
new file mode 100644
index 0000000..73e3c2c
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/logs-collector.sh
@@ -0,0 +1,102 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+echo "[INFO] Running Logs collector service"
+
+if [ -z "$1" ]; then
+    echo "[ERROR] Local logs directory doesn't specified"
+    exit 1
+fi
+
+echo "[INFO] Local logs directory: $1"
+
+if [ -z "$2" ]; then
+    echo "[ERROR] S3 folder where to upload logs doesn't specified"
+    exit 1
+fi
+
+echo "[INFO] S3 logs upload folder: $2"
+
+if [ -z "$3" ]; then
+    echo "[ERROR] Logs collection S3 trigger URL doesn't specified"
+    exit 1
+fi
+
+echo "[INFO] Logs collection S3 trigger URL: $3"
+
+echo "--------------------------------------------------------------------"
+
+TRIGGER_STATE=
+
+while true; do
+    sleep 1m
+
+    STATE=$(aws s3 ls $3)
+
+    if [ -z "$STATE" ] || [ "$STATE" == "$TRIGGER_STATE" ]; then
+        continue
+    fi
+
+    TRIGGER_STATE=$STATE
+
+    exists=
+    if [ -d "$1" ]; then
+        exists="true"
+    fi
+
+    echo "[INFO] Uploading logs from $1 to $2"
+
+    if [ "$exists" != "true" ]; then
+        echo "[INFO] Local logs directory $1 doesn't exist, thus there is nothing to upload"
+    fi
+
+    echo "--------------------------------------------------------------------"
+
+    if [ "$exists" != "true" ]; then
+        continue
+    fi
+
+    aws s3 sync --sse AES256 --delete "$1" "$2"
+
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to upload logs from $1 to $2 from first attempt"
+        sleep 30s
+
+        aws s3 sync --sse AES256 --delete "$1" "$2"
+
+        if [ $? -ne 0 ]; then
+            echo "[ERROR] Failed to upload logs from $1 to $2 from second attempt"
+            sleep 1m
+
+            aws s3 sync --sse AES256 --delete "$1" "$2"
+
+            if [ $? -ne 0 ]; then
+                echo "[ERROR] Failed to upload logs from $1 to $2 from third attempt"
+            else
+                echo "[INFO] Logs successfully uploaded from $1 to $2 from third attempt"
+            fi
+        else
+            echo "[INFO] Logs successfully uploaded from $1 to $2 from second attempt"
+        fi
+    else
+        echo "[INFO] Logs successfully uploaded from $1 to $2"
+    fi
+
+    echo "--------------------------------------------------------------------"
+done

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml b/modules/cassandra/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml
new file mode 100644
index 0000000..53c33a2
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml
@@ -0,0 +1,173 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <bean id="loadBalancingPolicy" class="com.datastax.driver.core.policies.RoundRobinPolicy"/>
+
+    <util:list id="contactPoints" value-type="java.lang.String">
+        ${CASSANDRA_SEEDS}
+    </util:list>
+
+    <bean id="cassandraAdminDataSource" class="org.apache.ignite.cache.store.cassandra.datasource.DataSource">
+        <property name="user" value="cassandra"/>
+        <property name="password" value="cassandra"/>
+        <property name="contactPoints" ref="contactPoints"/>
+        <property name="readConsistency" value="ONE"/>
+        <property name="writeConsistency" value="ONE"/>
+        <property name="loadBalancingPolicy" ref="loadBalancingPolicy"/>
+    </bean>
+
+    <!-- Persistence settings for 'cache1' -->
+    <bean id="cache1_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="primitive_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" column="key"/>
+    <valuePersistence class="java.lang.Long" strategy="PRIMITIVE" column="value"/>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Persistence settings for 'cache2' -->
+    <bean id="cache2_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="blob_test1">
+    <keyPersistence class="java.lang.Long" strategy="PRIMITIVE" />
+    <valuePersistence strategy="BLOB"/>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Persistence settings for 'cache3' -->
+    <bean id="cache3_settings" class="org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings">
+        <constructor-arg type="java.lang.String">
+            <value><![CDATA[
+<persistence keyspace="test1" table="pojo_test3" ttl="86400">
+    <keyspaceOptions>
+        REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 3}
+        AND DURABLE_WRITES = true
+    </keyspaceOptions>
+    <tableOptions>
+        comment = 'A most excellent and useful table'
+        AND read_repair_chance = 0.2
+    </tableOptions>
+    <keyPersistence class="org.apache.ignite.tests.pojos.PersonId" strategy="POJO">
+        <partitionKey>
+            <field name="companyCode" column="company" />
+            <field name="departmentCode" column="department" />
+        </partitionKey>
+        <clusterKey>
+            <field name="personNumber" column="number" sort="desc"/>
+        </clusterKey>
+    </keyPersistence>
+    <valuePersistence class="org.apache.ignite.tests.pojos.Person"
+                      strategy="POJO"
+                      serializer="org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer">
+        <field name="firstName" column="first_name" />
+        <field name="lastName" column="last_name" />
+        <field name="age" />
+        <field name="married" index="true"/>
+        <field name="height" />
+        <field name="weight" />
+        <field name="birthDate" column="birth_date" />
+        <field name="phones" />
+    </valuePersistence>
+</persistence>]]>
+            </value>
+        </constructor-arg>
+    </bean>
+
+    <!-- Ignite configuration -->
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- Turn on client mode -->
+        <property name="clientMode" value="true"/>
+
+        <property name="metricsLogFrequency" value="0"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <!-- Configuring persistence for "cache1" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="writeBehindEnabled" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache1_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache2" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="writeBehindEnabled" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache2_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "cache3" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache3"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="writeBehindEnabled" value="true"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="cache3_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Configuring remote ignite cluster connections -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                ${IGNITE_SEEDS}
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/test/bootstrap/aws/tests/tests-bootstrap.sh
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/test/bootstrap/aws/tests/tests-bootstrap.sh b/modules/cassandra/src/test/bootstrap/aws/tests/tests-bootstrap.sh
new file mode 100644
index 0000000..d00ddb6
--- /dev/null
+++ b/modules/cassandra/src/test/bootstrap/aws/tests/tests-bootstrap.sh
@@ -0,0 +1,379 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+S3_ROOT=s3://bucket/folder
+S3_DOWNLOADS=$S3_ROOT/test
+S3_SYSTEM=$S3_ROOT/test1
+
+TESTS_PACKAGE_DONLOAD_URL=$S3_DOWNLOADS/ignite-cassandra-tests-1.6.0-SNAPSHOT.zip
+TESTS_PACKAGE_ZIP=ignite-cassandra-tests-1.6.0-SNAPSHOT.zip
+TESTS_PACKAGE_UNZIP_DIR=ignite-cassandra-tests
+
+S3_LOGS_URL=$S3_SYSTEM/logs/t-logs
+S3_LOGS_TRIGGER_URL=$S3_SYSTEM/logs-trigger
+S3_IGNITE_NODES_DISCOVERY_URL=$S3_SYSTEM/i-discovery
+S3_CASSANDRA_NODES_DISCOVERY_URL=$S3_SYSTEM/c-discovery
+S3_TEST_NODES_DISCOVERY_URL=$S3_SYSTEM/t-discovery
+S3_TESTS_SUCCESS_URL=$S3_SYSTEM/t-success
+S3_TESTS_FAILURE_URL=$S3_SYSTEM/t-failure
+S3_TESTS_RUNNING_URL=$S3_SYSTEM/t-running
+S3_TESTS_WAITING_URL=$S3_SYSTEM/t-waiting
+S3_IGNITE_SUCCESS_URL=$S3_SYSTEM/i-success
+S3_IGNITE_FAILURE_URL=$S3_SYSTEM/i-failure
+S3_CASSANDRA_SUCCESS_URL=$S3_SYSTEM/c-success
+S3_CASSANDRA_FAILURE_URL=$S3_SYSTEM/c-failure
+S3_TESTS_FIRST_NODE_LOCK_URL=$S3_SYSTEM/t-first-node-lock
+S3_TESTS_SUMMARY_URL=$S3_SYSTEM/t-summary.zip
+
+INSTANCE_REGION=us-west-2
+INSTANCE_NAME_TAG=TEST-SERVER
+INSTANCE_OWNER_TAG=ignite@apache.org
+INSTANCE_PROJECT_TAG=ignite
+
+CASSANDRA_NODES_COUNT=50
+IGNITE_NODES_COUNT=30
+TEST_NODES_COUNT=30
+
+TESTS_TYPE="ignite"
+
+terminate()
+{
+    if [[ "$S3_TESTS_SUCCESS_URL" != */ ]]; then
+        S3_TESTS_SUCCESS_URL=${S3_TESTS_SUCCESS_URL}/
+    fi
+
+    if [[ "$S3_TESTS_FAILURE_URL" != */ ]]; then
+        S3_TESTS_FAILURE_URL=${S3_TESTS_FAILURE_URL}/
+    fi
+
+    host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+    msg=$host_name
+
+    if [ -n "$1" ]; then
+        echo "[ERROR] $1"
+        echo "[ERROR]-----------------------------------------------------"
+        echo "[ERROR] Test node bootstrap failed"
+        echo "[ERROR]-----------------------------------------------------"
+        msg=$1
+        reportFolder=${S3_TESTS_FAILURE_URL}${host_name}
+        reportFile=$reportFolder/__error__
+    else
+        echo "[INFO]-----------------------------------------------------"
+        echo "[INFO] Test node bootstrap successfully completed"
+        echo "[INFO]-----------------------------------------------------"
+        reportFolder=${S3_TESTS_SUCCESS_URL}${host_name}
+        reportFile=$reportFolder/__success__
+    fi
+
+    echo $msg > /opt/bootstrap-result
+
+    aws s3 rm --recursive $reportFolder
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed drop report folder: $reportFolder"
+    fi
+
+    aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+    if [ $? -ne 0 ]; then
+        echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+    fi
+
+    rm -f /opt/bootstrap-result
+
+    if [ -n "$1" ]; then
+        exit 1
+    fi
+
+    exit 0
+}
+
+tagInstance()
+{
+    export EC2_HOME=/opt/aws/apitools/ec2
+    export JAVA_HOME=/opt/jdk1.8.0_77
+    export PATH=$JAVA_HOME/bin:$EC2_HOME/bin:$PATH
+
+    INSTANCE_ID=$(curl http://169.254.169.254/latest/meta-data/instance-id)
+    if [ $? -ne 0 ]; then
+        terminate "Failed to get instance metadata to tag it"
+    fi
+
+    if [ -n "$INSTANCE_NAME_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag Name=${INSTANCE_NAME_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: Name=${INSTANCE_NAME_TAG}"
+        fi
+    fi
+
+    if [ -n "$INSTANCE_OWNER_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag owner=${INSTANCE_OWNER_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: owner=${INSTANCE_OWNER_TAG}"
+        fi
+    fi
+
+    if [ -n "$INSTANCE_PROJECT_TAG" ]; then
+        ec2-create-tags $INSTANCE_ID --tag project=${INSTANCE_PROJECT_TAG} --region $INSTANCE_REGION
+        if [ $code -ne 0 ]; then
+            terminate "Failed to tag EC2 instance with: project=${INSTANCE_PROJECT_TAG}"
+        fi
+    fi
+}
+
+downloadPackage()
+{
+    echo "[INFO] Downloading $3 package from $1 into $2"
+
+    if [[ "$1" == s3* ]]; then
+        aws s3 cp $1 $2
+
+        if [ $? -ne 0 ]; then
+            echo "[WARN] Failed to download $3 package from first attempt"
+            rm -Rf $2
+            sleep 10s
+
+            echo "[INFO] Trying second attempt to download $3 package"
+            aws s3 cp $1 $2
+
+            if [ $? -ne 0 ]; then
+                echo "[WARN] Failed to download $3 package from second attempt"
+                rm -Rf $2
+                sleep 10s
+
+                echo "[INFO] Trying third attempt to download $3 package"
+                aws s3 cp $1 $2
+
+                if [ $? -ne 0 ]; then
+                    terminate "All three attempts to download $3 package from $1 are failed"
+                fi
+            fi
+        fi
+    else
+        curl "$1" -o "$2"
+
+        if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+            echo "[WARN] Failed to download $3 package from first attempt"
+            rm -Rf $2
+            sleep 10s
+
+            echo "[INFO] Trying second attempt to download $3 package"
+            curl "$1" -o "$2"
+
+            if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+                echo "[WARN] Failed to download $3 package from second attempt"
+                rm -Rf $2
+                sleep 10s
+
+                echo "[INFO] Trying third attempt to download $3 package"
+                curl "$1" -o "$2"
+
+                if [ $? -ne 0 ] && [ $? -ne 6 ]; then
+                    terminate "All three attempts to download $3 package from $1 are failed"
+                fi
+            fi
+        fi
+    fi
+
+    echo "[INFO] $3 package successfully downloaded from $1 into $2"
+}
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Tests node"
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Tests type: $TESTS_TYPE"
+echo "[INFO] Test nodes count: $TEST_NODES_COUNT"
+echo "[INFO] Ignite nodes count: $IGNITE_NODES_COUNT"
+echo "[INFO] Cassandra nodes count: $CASSANDRA_NODES_COUNT"
+echo "[INFO] Tests summary URL: $S3_TESTS_SUMMARY_URL"
+echo "[INFO] Tests first node lock URL: $S3_TESTS_FIRST_NODE_LOCK_URL"
+echo "[INFO] Logs URL: $S3_LOGS_URL"
+echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER_URL"
+echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+echo "[INFO] Test node discovery URL: $S3_TEST_NODES_DISCOVERY_URL"
+echo "[INFO] Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY_URL"
+echo "[INFO] Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY_URL"
+echo "[INFO] Tests running URL: $S3_TESTS_RUNNING_URL"
+echo "[INFO] Tests waiting URL: $S3_TESTS_WAITING_URL"
+echo "[INFO] Tests success URL: $S3_TESTS_SUCCESS_URL"
+echo "[INFO] Tests failure URL: $S3_TESTS_FAILURE_URL"
+echo "[INFO] Ignite success URL: $S3_IGNITE_SUCCESS_URL"
+echo "[INFO] Ignite failure URL: $S3_IGNITE_FAILURE_URL"
+echo "[INFO] Cassandra success URL: $S3_CASSANDRA_SUCCESS_URL"
+echo "[INFO] Cassandra failure URL: $S3_CASSANDRA_FAILURE_URL"
+echo "[INFO]-----------------------------------------------------------------"
+
+echo "[INFO] Installing 'wget' package"
+yum -y install wget
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'wget' package"
+fi
+
+echo "[INFO] Installing 'net-tools' package"
+yum -y install net-tools
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'net-tools' package"
+fi
+
+echo "[INFO] Installing 'python' package"
+yum -y install python
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'python' package"
+fi
+
+echo "[INFO] Installing 'unzip' package"
+yum -y install unzip
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'unzip' package"
+fi
+
+rm -Rf /opt/jdk1.8.0_77 /opt/jdk-8u77-linux-x64.tar.gz
+
+echo "[INFO] Downloading 'jdk-8u77'"
+wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz" -O /opt/jdk-8u77-linux-x64.tar.gz
+if [ $? -ne 0 ]; then
+    terminate "Failed to download 'jdk-8u77'"
+fi
+
+echo "[INFO] Unzipping 'jdk-8u77'"
+tar -xvzf /opt/jdk-8u77-linux-x64.tar.gz -C /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to untar 'jdk-8u77'"
+fi
+
+rm -Rf /opt/jdk-8u77-linux-x64.tar.gz
+
+downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+echo "[INFO] Installing 'pip'"
+python /opt/get-pip.py
+if [ $? -ne 0 ]; then
+    terminate "Failed to install 'pip'"
+fi
+
+echo "[INFO] Installing 'awscli'"
+pip install --upgrade awscli
+if [ $? -ne 0 ]; then
+    echo "[ERROR] Failed to install 'awscli' using pip"
+    echo "[INFO] Trying to install awscli using zip archive"
+    echo "[INFO] Downloading awscli zip"
+
+    downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+    echo "[INFO] Unzipping awscli zip"
+    unzip /opt/awscli-bundle.zip -d /opt
+    if [ $? -ne 0 ]; then
+        terminate "Failed to unzip awscli zip"
+    fi
+
+    rm -fR /opt/awscli-bundle.zip
+
+    echo "[INFO] Installing awscli"
+    /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+    if [ $? -ne 0 ]; then
+        terminate "Failed to install awscli"
+    fi
+
+    echo "[INFO] Successfully installed awscli from zip archive"
+fi
+
+tagInstance
+
+echo "[INFO] Creating 'ignite' group"
+exists=$(cat /etc/group | grep ignite)
+if [ -z "$exists" ]; then
+    groupadd ignite
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create 'ignite' group"
+    fi
+fi
+
+echo "[INFO] Creating 'ignite' user"
+exists=$(cat /etc/passwd | grep ignite)
+if [ -z "$exists" ]; then
+    useradd -g ignite ignite
+    if [ $? -ne 0 ]; then
+        terminate "Failed to create 'ignite' user"
+    fi
+fi
+
+downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/$TESTS_PACKAGE_ZIP" "Tests"
+
+unzip /opt/$TESTS_PACKAGE_ZIP -d /opt
+if [ $? -ne 0 ]; then
+    terminate "Failed to unzip tests package: $TESTS_PACKAGE_DONLOAD_URL"
+fi
+
+mv /opt/$TESTS_PACKAGE_UNZIP_DIR /opt/ignite-cassandra-tests
+
+if [ ! -f "/opt/ignite-cassandra-tests/cassandra-load-tests.sh" ]; then
+    terminate "There are no cassandra-load-tests.sh in tests package"
+fi
+
+if [ ! -f "/opt/ignite-cassandra-tests/ignite-load-tests.sh" ]; then
+    terminate "There are no ignite-load-tests.sh in tests package"
+fi
+
+if [ ! -f "/opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template.xml" ]; then
+    terminate "There are no ignite-cassandra-client-template.xml in tests package"
+fi
+
+if [ ! -f "/opt/$TESTS_PACKAGE_UNZIP_DIR/bootstrap/aws/logs-collector.sh" ]; then
+    terminate "There are no logs-collector.sh in tests package"
+fi
+
+chown -R ignite:ignite /opt/ignite-cassandra-tests
+find /opt/ignite-cassandra-tests -type f -name "*.sh" -exec chmod ug+x {} \;
+
+cp -f /opt/ignite-cassandra-tests/bootstrap/aws/logs-collector.sh /opt
+chown -R ignite:ignite /opt/logs-collector.sh
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+echo "export JAVA_HOME=/opt/jdk1.8.0_77" >> $profile
+echo "export PATH=\$JAVA_HOME/bin:\IGNITE_HOME/bin:\$PATH" >> $profile
+echo "export TESTS_TYPE=$TESTS_TYPE" >> $profile
+echo "export S3_TESTS_SUMMARY_URL=$S3_TESTS_SUMMARY_URL" >> $profile
+echo "export S3_CASSANDRA_NODES_DISCOVERY_URL=$S3_CASSANDRA_NODES_DISCOVERY_URL" >> $profile
+echo "export S3_TEST_NODES_DISCOVERY_URL=$S3_TEST_NODES_DISCOVERY_URL" >> $profile
+echo "export S3_IGNITE_NODES_DISCOVERY_URL=$S3_IGNITE_NODES_DISCOVERY_URL" >> $profile
+echo "export S3_TESTS_RUNNING_URL=$S3_TESTS_RUNNING_URL" >> $profile
+echo "export S3_TESTS_WAITING_URL=$S3_TESTS_WAITING_URL" >> $profile
+echo "export S3_TESTS_SUCCESS_URL=$S3_TESTS_SUCCESS_URL" >> $profile
+echo "export S3_TESTS_FAILURE_URL=$S3_TESTS_FAILURE_URL" >> $profile
+echo "export S3_IGNITE_SUCCESS_URL=$S3_IGNITE_SUCCESS_URL" >> $profile
+echo "export S3_IGNITE_FAILURE_URL=$S3_IGNITE_FAILURE_URL" >> $profile
+echo "export S3_CASSANDRA_SUCCESS_URL=$S3_CASSANDRA_SUCCESS_URL" >> $profile
+echo "export S3_CASSANDRA_FAILURE_URL=$S3_CASSANDRA_FAILURE_URL" >> $profile
+echo "export S3_TESTS_FIRST_NODE_LOCK_URL=$S3_TESTS_FIRST_NODE_LOCK_URL" >> $profile
+echo "export CASSANDRA_NODES_COUNT=$CASSANDRA_NODES_COUNT" >> $profile
+echo "export IGNITE_NODES_COUNT=$IGNITE_NODES_COUNT" >> $profile
+echo "export TEST_NODES_COUNT=$TEST_NODES_COUNT" >> $profile
+echo "export S3_LOGS_TRIGGER_URL=$S3_LOGS_TRIGGER_URL" >> $profile
+
+HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+/opt/logs-collector.sh "/opt/ignite-cassandra-tests/logs" "$S3_LOGS_URL/$HOST_NAME" "$S3_LOGS_TRIGGER_URL" > /opt/ignite-cassandra-tests/logs-collector.log &
+
+cmd="/opt/ignite-cassandra-tests/bootstrap/aws/tests/tests-run.sh"
+
+#sudo -u ignite -g ignite sh -c "$cmd | tee /opt/ignite-cassandra-tests/start.log"
+
+$cmd | tee /opt/ignite-cassandra-tests/start.log
\ No newline at end of file


[48/50] ignite git commit: Revert "IGNITE-3331: IGFS: Better affinity management for client tasks."

Posted by vo...@apache.org.
Revert "IGNITE-3331: IGFS: Better affinity management for client tasks."

This reverts commit 06831b13f4a9693e767f2e299abcc57605c87da5.


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

Branch: refs/heads/ignite-3341
Commit: c952def5de5f2df1e825869d99d362af71ba87ab
Parents: 04af846
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 17:39:46 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 17:39:46 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 45 ++++----------------
 .../igfs/client/IgfsClientAbstractCallable.java |  5 ---
 .../igfs/client/IgfsClientAffinityCallable.java |  5 ---
 .../igfs/client/IgfsClientDeleteCallable.java   |  5 ---
 .../igfs/client/IgfsClientExistsCallable.java   |  5 ---
 .../igfs/client/IgfsClientInfoCallable.java     |  5 ---
 .../client/IgfsClientListFilesCallable.java     |  5 ---
 .../client/IgfsClientListPathsCallable.java     |  5 ---
 .../igfs/client/IgfsClientMkdirsCallable.java   |  5 ---
 .../igfs/client/IgfsClientRenameCallable.java   |  5 ---
 .../igfs/client/IgfsClientSetTimesCallable.java |  5 ---
 .../igfs/client/IgfsClientSizeCallable.java     |  5 ---
 .../igfs/client/IgfsClientSummaryCallable.java  |  5 ---
 .../igfs/client/IgfsClientUpdateCallable.java   |  5 ---
 .../meta/IgfsClientMetaIdsForPathCallable.java  |  5 ---
 .../meta/IgfsClientMetaInfoForPathCallable.java |  5 ---
 16 files changed, 9 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 30c7d10..943c4c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -150,11 +150,8 @@ public class IgfsMetaManager extends IgfsManager {
     /** Client flag. */
     private final boolean client;
 
-    /** Compute facade. */
-    private IgniteCompute compute;
-
     /** Compute facade for client tasks. */
-    private IgniteCompute metaCompute;
+    private IgniteCompute cliCompute;
 
     /**
      * Constructor.
@@ -251,12 +248,7 @@ public class IgfsMetaManager extends IgfsManager {
      */
     <T> T runClientTask(IgfsClientAbstractCallable<T> task) {
         try {
-            if (!task.isReadOnly() && cfg.isColocateMetadata())
-                // If task mutates state and co-location is enabled, we route request to primary node.
-                return compute().affinityCall(cfg.getMetaCacheName(), IgfsUtils.ROOT_ID, task);
-            else
-                // Otherwise we route to any available data node.
-                return metaCompute().call(task);
+            return clientCompute().call(task);
         }
         catch (ClusterTopologyException e) {
             throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e);
@@ -268,43 +260,24 @@ public class IgfsMetaManager extends IgfsManager {
      *
      * @return Compute facade.
      */
-    private IgniteCompute compute() {
-        IgniteCompute compute0 = compute;
-
-        if (compute0 == null) {
-            compute0 = igfsCtx.kernalContext().grid().compute();
-
-            compute = compute0;
-        }
-
-        assert compute0 != null;
-
-        return compute0;
-    }
-
-    /**
-     * Get metadata compute facade for client tasks.
-     *
-     * @return Metadata compute facade.
-     */
-    private IgniteCompute metaCompute() {
+    private IgniteCompute clientCompute() {
         assert client;
 
-        IgniteCompute metaCompute0 = metaCompute;
+        IgniteCompute cliCompute0 = cliCompute;
 
-        if (metaCompute0 == null) {
+        if (cliCompute0 == null) {
             IgniteEx ignite = igfsCtx.kernalContext().grid();
 
             ClusterGroup cluster = ignite.cluster().forIgfsMetadataDataNodes(cfg.getName(), cfg.getMetaCacheName());
 
-            metaCompute0 = ignite.compute(cluster);
+            cliCompute0 = ignite.compute(cluster);
 
-            metaCompute = metaCompute0;
+            cliCompute = cliCompute0;
         }
 
-        assert metaCompute0 != null;
+        assert cliCompute0 != null;
 
-        return metaCompute0;
+        return cliCompute0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
index c06619f..d9c3456 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAbstractCallable.java
@@ -85,11 +85,6 @@ public abstract class IgfsClientAbstractCallable<T> implements IgniteCallable<T>
      */
     protected abstract T call0(IgfsContext ctx) throws Exception;
 
-    /**
-     * @return {@code True} if task is read-only and does not change metadata.
-     */
-    public abstract boolean isReadOnly();
-
     /** {@inheritDoc} */
     @Override public final void writeBinary(BinaryWriter writer) throws BinaryObjectException {
         BinaryRawWriter rawWriter = writer.rawWriter();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
index 6e2748d..1668f36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientAffinityCallable.java
@@ -74,11 +74,6 @@ public class IgfsClientAffinityCallable extends IgfsClientAbstractCallable<Colle
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeLong(start);
         writer.writeLong(len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
index 21f8c46..c1b8be8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientDeleteCallable.java
@@ -61,11 +61,6 @@ public class IgfsClientDeleteCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeBoolean(recursive);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
index 536cdc8..04b63d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientExistsCallable.java
@@ -52,11 +52,6 @@ public class IgfsClientExistsCallable extends IgfsClientAbstractCallable<Boolean
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientExistsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
index 6bab8b8..f97c3c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientInfoCallable.java
@@ -53,11 +53,6 @@ public class IgfsClientInfoCallable extends IgfsClientAbstractCallable<IgfsFile>
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientInfoCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
index deb8836..325e714 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListFilesCallable.java
@@ -55,11 +55,6 @@ public class IgfsClientListFilesCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListFilesCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
index ad6c858..78b4c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientListPathsCallable.java
@@ -54,11 +54,6 @@ public class IgfsClientListPathsCallable extends IgfsClientAbstractCallable<Coll
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientListPathsCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
index 6ecb6ea..944da6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientMkdirsCallable.java
@@ -66,11 +66,6 @@ public class IgfsClientMkdirsCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         IgfsUtils.writeProperties(writer, props);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
index 142af47..55afb83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientRenameCallable.java
@@ -64,11 +64,6 @@ public class IgfsClientRenameCallable extends IgfsClientAbstractCallable<Void> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         IgfsUtils.writePath(writer, destPath);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
index 8c302bc..277effc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSetTimesCallable.java
@@ -69,11 +69,6 @@ public class IgfsClientSetTimesCallable extends IgfsClientAbstractCallable<Void>
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         writer.writeLong(accessTime);
         writer.writeLong(modificationTime);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
index 29a7b7a..474a940 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSizeCallable.java
@@ -53,11 +53,6 @@ public class IgfsClientSizeCallable extends IgfsClientAbstractCallable<Long> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSizeCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
index 1e2795c..7e29029 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientSummaryCallable.java
@@ -53,11 +53,6 @@ public class IgfsClientSummaryCallable extends IgfsClientAbstractCallable<IgfsPa
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientSummaryCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
index 82d6e8d..4acf4eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/IgfsClientUpdateCallable.java
@@ -65,11 +65,6 @@ public class IgfsClientUpdateCallable extends IgfsClientAbstractCallable<IgfsFil
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
     @Override public void writeBinary0(BinaryRawWriter writer) throws BinaryObjectException {
         IgfsUtils.writeProperties(writer, props);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
index b07a0cb..7b3d142 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaIdsForPathCallable.java
@@ -59,11 +59,6 @@ public class IgfsClientMetaIdsForPathCallable extends IgfsClientAbstractCallable
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaIdsForPathCallable.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c952def5/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
index e7e0164..cb31663 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/client/meta/IgfsClientMetaInfoForPathCallable.java
@@ -57,11 +57,6 @@ public class IgfsClientMetaInfoForPathCallable extends IgfsClientAbstractCallabl
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isReadOnly() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsClientMetaInfoForPathCallable.class, this);
     }


[35/50] ignite git commit: IGNITE-3216 Need to deduplicate addresses registered in the IP finder

Posted by vo...@apache.org.
IGNITE-3216 Need to deduplicate addresses registered in the IP finder


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

Branch: refs/heads/ignite-3341
Commit: a4bbf0b70349fee39689b72c10e1c3dc26363c3f
Parents: 5c3efa9
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Jun 16 10:25:38 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Jun 16 10:25:38 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |  8 +--
 .../ignite/internal/util/lang/GridFunc.java     | 64 --------------------
 .../internal/util/IgniteUtilsSelfTest.java      | 15 +++++
 3 files changed, 19 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a4bbf0b7/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index c898909..faf026b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -8550,7 +8550,7 @@ public abstract class IgniteUtils {
      */
     public static Collection<InetAddress> toInetAddresses(Collection<String> addrs,
         Collection<String> hostNames) throws IgniteCheckedException {
-        List<InetAddress> res = new ArrayList<>(addrs.size());
+        Set<InetAddress> res = new HashSet<>(addrs.size());
 
         Iterator<String> hostNamesIt = hostNames.iterator();
 
@@ -8583,7 +8583,7 @@ public abstract class IgniteUtils {
             throw new IgniteCheckedException("Addresses can not be resolved [addr=" + addrs +
                 ", hostNames=" + hostNames + ']');
 
-        return F.viewListReadOnly(res, F.<InetAddress>identity());
+        return res;
     }
 
     /**
@@ -8609,7 +8609,7 @@ public abstract class IgniteUtils {
      */
     public static Collection<InetSocketAddress> toSocketAddresses(Collection<String> addrs,
         Collection<String> hostNames, int port) {
-        List<InetSocketAddress> res = new ArrayList<>(addrs.size());
+        Set<InetSocketAddress> res = new HashSet<>(addrs.size());
 
         Iterator<String> hostNamesIt = hostNames.iterator();
 
@@ -8630,7 +8630,7 @@ public abstract class IgniteUtils {
             res.add(new InetSocketAddress(addr, port));
         }
 
-        return F.viewListReadOnly(res, F.<InetSocketAddress>identity());
+        return res;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4bbf0b7/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index 8eeca6b..20e31c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1695,70 +1695,6 @@ public class GridFunc {
     }
 
     /**
-     * Creates read-only light-weight view on given list with provided transformation.
-     * Resulting list will only "have" {@code transformed} elements. Note that only wrapping
-     * list will be created and no duplication of data will occur.
-     *
-     * @param c Input list that serves as a base for the view.
-     * @param trans Transformation closure.
-     * @param <T1> Type of the list.
-     * @return Light-weight view on given list with provided transformation.
-     */
-    @SuppressWarnings("RedundantTypeArguments")
-    public static <T1, T2> List<T2> viewListReadOnly(@Nullable final List<? extends T1> c,
-        final IgniteClosure<? super T1, T2> trans) {
-        A.notNull(trans, "trans");
-
-        if (isEmpty(c))
-            return Collections.emptyList();
-
-        assert c != null;
-
-        return new GridSerializableList<T2>() {
-            /** */
-            private static final long serialVersionUID = 3126625219739967068L;
-
-            @Override public T2 get(int idx) {
-                return trans.apply(c.get(idx));
-            }
-
-            @NotNull
-            @Override public Iterator<T2> iterator() {
-                return F.<T1, T2>iterator(c, trans, true);
-            }
-
-            @Override public int size() {
-                return c.size();
-            }
-
-            @Override public boolean isEmpty() {
-                return c.isEmpty();
-            }
-        };
-    }
-
-    /**
-     * Creates a view on given list with provided transformer and predicates.
-     * Resulting list will only "have" elements for which all provided predicates, if any,
-     * evaluate to {@code true}. Note that a new collection will be created and data will
-     * be copied.
-     *
-     * @param c Input list that serves as a base for the view.
-     * @param trans Transforming closure from T1 to T2.
-     * @param p Optional predicates. If predicates are not provided - all elements will be in the view.
-     * @return View on given list with provided predicate.
-     */
-    public static <T1, T2> List<T2> transformList(Collection<? extends T1> c,
-        IgniteClosure<? super T1, T2> trans, @Nullable IgnitePredicate<? super T1>... p) {
-        A.notNull(c, "c", trans, "trans");
-
-        if (isAlwaysFalse(p))
-            return Collections.emptyList();
-
-        return new ArrayList<>(transform(retain(c, true, p), trans));
-    }
-
-    /**
      * Creates light-weight view on given map with provided predicates. Resulting map will
      * only "have" keys for which all provided predicates, if any, evaluates to {@code true}.
      * Note that only wrapping map will be created and no duplication of data will occur.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4bbf0b7/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
index 520fa76..d774065 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/IgniteUtilsSelfTest.java
@@ -742,6 +742,21 @@ public class IgniteUtilsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testToSocketAddressesNoDuplicates() {
+        Collection<String> addrs = new ArrayList<>();
+
+        addrs.add("127.0.0.1");
+        addrs.add("localhost");
+
+        Collection<String> hostNames = new ArrayList<>();
+        int port = 1234;
+
+        assertEquals(1, U.toSocketAddresses(addrs, hostNames, port).size());
+    }
+
+    /**
      * Test enum.
      */
     private enum TestEnum {


[05/50] ignite git commit: ignite-3038 Do not use custom discovery events to start continuous queries for system caches

Posted by vo...@apache.org.
ignite-3038 Do not use custom discovery events to start continuous queries for system caches


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

Branch: refs/heads/ignite-3341
Commit: 7f878c56cf4e63cff1773dfe834cda7cd91c62ac
Parents: e10ffef
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 14 10:17:50 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 14 10:17:50 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  |  51 ++-
 .../processors/cache/GridCacheAdapter.java      |  12 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  29 +-
 .../continuous/CacheContinuousQueryManager.java | 135 +++++++
 .../cacheobject/IgniteCacheObjectProcessor.java |   7 +
 .../IgniteCacheObjectProcessorImpl.java         |   5 +
 .../continuous/GridContinuousProcessor.java     | 217 +++++++---
 .../service/GridServiceProcessor.java           | 391 ++++++++++---------
 ...eClientReconnectContinuousProcessorTest.java |  60 ++-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   2 +-
 .../IgniteNoCustomEventsOnNodeStart.java        |  80 ++++
 .../service/GridServiceClientNodeTest.java      | 102 ++++-
 .../testsuites/IgniteCacheTestSuite2.java       |   3 +
 13 files changed, 813 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 8f566a9..b4c9607 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException;
 import org.apache.ignite.internal.util.GridStripedLock;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.PluginProvider;
 
@@ -64,6 +65,9 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
     /** Non-volatile on purpose. */
     private int failedCnt;
 
+    /** */
+    private ContinuousQueryListener lsnr;
+
     /**
      * @param plugins Plugins.
      * @throws IgniteCheckedException In case of error.
@@ -75,25 +79,58 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
     }
 
     /**
+     * @param ctx Context.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException {
+        if (ctx.clientNode()) {
+            lsnr = new ContinuousQueryListener(ctx.log(MarshallerContextImpl.class), workDir);
+
+            ctx.continuous().registerStaticRoutine(
+                CU.MARSH_CACHE_NAME,
+                lsnr,
+                null,
+                null);
+        }
+    }
+
+    /**
      * @param ctx Kernal context.
      * @throws IgniteCheckedException In case of error.
      */
     public void onMarshallerCacheStarted(GridKernalContext ctx) throws IgniteCheckedException {
         assert ctx != null;
 
-        if (!ctx.isDaemon()) {
+        log = ctx.log(MarshallerContextImpl.class);
+
+        cache = ctx.cache().marshallerCache();
+
+        if (ctx.cache().marshallerCache().context().affinityNode()) {
             ctx.cache().marshallerCache().context().continuousQueries().executeInternalQuery(
-                new ContinuousQueryListener(ctx.log(MarshallerContextImpl.class), workDir),
+                new ContinuousQueryListener(log, workDir),
                 null,
-                ctx.cache().marshallerCache().context().affinityNode(),
+                true,
                 true,
                 false
             );
         }
-
-        log = ctx.log(MarshallerContextImpl.class);
-
-        cache = ctx.cache().marshallerCache();
+        else {
+            if (lsnr != null) {
+                ctx.closure().runLocalSafe(new Runnable() {
+                    @SuppressWarnings("unchecked")
+                    @Override public void run() {
+                        try {
+                            Iterable entries = cache.context().continuousQueries().existingEntries(false, null);
+
+                            lsnr.onUpdated(entries);
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to load marshaller cache entries: " + e, e);
+                        }
+                    }
+                });
+            }
+        }
 
         latch.countDown();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 6e647c5..b4daec2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -4138,8 +4138,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /**
      * @return Distributed ignite cache iterator.
+     * @throws IgniteCheckedException If failed.
      */
     public Iterator<Cache.Entry<K, V>> igniteIterator() throws IgniteCheckedException {
+        return igniteIterator(ctx.keepBinary());
+    }
+
+    /**
+     * @param keepBinary
+     * @return Distributed ignite cache iterator.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Iterator<Cache.Entry<K, V>> igniteIterator(boolean keepBinary) throws IgniteCheckedException {
         GridCacheContext ctx0 = ctx.isNear() ? ctx.near().dht().context() : ctx;
 
         final CacheOperationContext opCtx = ctx.operationContextPerCall();
@@ -4147,7 +4157,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (!ctx0.isSwapOrOffheapEnabled() && ctx0.kernalContext().discovery().size() == 1)
             return localIteratorHonorExpirePolicy(opCtx);
 
-        final GridCloseableIterator<Map.Entry<K, V>> iter = ctx0.queries().createScanQuery(null, null, ctx.keepBinary())
+        final GridCloseableIterator<Map.Entry<K, V>> iter = ctx0.queries().createScanQuery(null, null, keepBinary)
             .keepAll(false)
             .executeScanQuery();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index fe73f8a..d5756a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -154,9 +154,6 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     /** Metadata updates collected before metadata cache is initialized. */
     private final Map<Integer, BinaryMetadata> metaBuf = new ConcurrentHashMap<>();
 
-    /** */
-    private UUID metaCacheQryId;
-
     /**
      * @param ctx Kernal context.
      */
@@ -260,6 +257,17 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /** {@inheritDoc} */
+    @Override public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException {
+        if (clientNode && !ctx.isDaemon()) {
+            ctx.continuous().registerStaticRoutine(
+                CU.UTILITY_CACHE_NAME,
+                new MetaDataEntryListener(),
+                new MetaDataEntryFilter(),
+                null);
+        }
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
         IgniteCacheProxy<Object, Object> proxy = ctx.cache().jcache(CU.UTILITY_CACHE_NAME);
@@ -276,13 +284,6 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         if (clientNode) {
             assert !metaDataCache.context().affinityNode();
 
-            metaCacheQryId = metaDataCache.context().continuousQueries().executeInternalQuery(
-                new MetaDataEntryListener(),
-                new MetaDataEntryFilter(),
-                false,
-                true,
-                false);
-
             while (true) {
                 ClusterNode oldestSrvNode =
                     CU.oldestAliveCacheServerNode(ctx.cache().context(), AffinityTopologyVersion.NONE);
@@ -363,14 +364,6 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public void onKernalStop(boolean cancel) {
-        super.onKernalStop(cancel);
-
-        if (metaCacheQryId != null)
-            metaDataCache.context().continuousQueries().cancelInternalQuery(metaCacheQryId);
-    }
-
     /**
      * @param key Metadata key.
      * @param newMeta Metadata.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index b042249..c966527 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -31,6 +31,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import javax.cache.Cache;
 import javax.cache.configuration.CacheEntryListenerConfiguration;
 import javax.cache.configuration.Factory;
 import javax.cache.event.CacheEntryCreatedListener;
@@ -46,6 +47,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.CacheQueryEntryEvent;
 import org.apache.ignite.cache.query.ContinuousQuery;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
@@ -57,8 +59,10 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateFuture;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteAsyncCallback;
 import org.apache.ignite.lang.IgniteClosure;
@@ -728,6 +732,70 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param keepBinary Keep binary flag.
+     * @param filter Filter.
+     * @return Iterable for events created for existing cache entries.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Iterable<CacheEntryEvent<?, ?>> existingEntries(final boolean keepBinary, final CacheEntryEventFilter filter)
+        throws IgniteCheckedException {
+        final Iterator<Cache.Entry<?, ?>> it = cctx.cache().igniteIterator(keepBinary);
+
+        final Cache cache = cctx.kernalContext().cache().jcache(cctx.name());
+
+        return new Iterable<CacheEntryEvent<?, ?>>() {
+            @Override public Iterator<CacheEntryEvent<?, ?>> iterator() {
+                return new Iterator<CacheEntryEvent<?, ?>>() {
+                    private CacheQueryEntryEvent<?, ?> next;
+
+                    {
+                        advance();
+                    }
+
+                    @Override public boolean hasNext() {
+                        return next != null;
+                    }
+
+                    @Override public CacheEntryEvent<?, ?> next() {
+                        if (!hasNext())
+                            throw new NoSuchElementException();
+
+                        CacheEntryEvent next0 = next;
+
+                        advance();
+
+                        return next0;
+                    }
+
+                    @Override public void remove() {
+                        throw new UnsupportedOperationException();
+                    }
+
+                    private void advance() {
+                        next = null;
+
+                        while (next == null) {
+                            if (!it.hasNext())
+                                break;
+
+                            Cache.Entry e = it.next();
+
+                            next = new CacheEntryEventImpl(
+                                cache,
+                                CREATED,
+                                e.getKey(),
+                                e.getValue());
+
+                            if (filter != null && !filter.evaluate(next))
+                                next = null;
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    /**
      * @param nodes Nodes.
      * @return {@code True} if all nodes greater than {@link GridContinuousProcessor#QUERY_MSG_VER_2_SINCE},
      *     otherwise {@code false}.
@@ -1129,4 +1197,71 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
                 lsnr.acknowledgeBackupOnTimeout(ctx);
         }
     }
+
+    /**
+     *
+     */
+    private static class CacheEntryEventImpl extends CacheQueryEntryEvent {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        @GridToStringInclude
+        private Object key;
+
+        /** */
+        @GridToStringInclude
+        private Object val;
+
+        /**
+         * @param src Event source.
+         * @param evtType Event type.
+         * @param key Key.
+         * @param val Value.
+         */
+        public CacheEntryEventImpl(Cache src, EventType evtType, Object key, Object val) {
+            super(src, evtType);
+
+            this.key = key;
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getPartitionUpdateCounter() {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object getOldValue() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isOldValueAvailable() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object getKey() {
+            return key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object getValue() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object unwrap(Class cls) {
+            if (cls.isAssignableFrom(getClass()))
+                return cls.cast(this);
+
+            throw new IllegalArgumentException("Unwrapping to class is not supported: " + cls);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(CacheEntryEventImpl.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index cadf1a9..99de507 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -22,6 +22,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridComponent;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessor;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -34,6 +35,12 @@ import org.jetbrains.annotations.Nullable;
  */
 public interface IgniteCacheObjectProcessor extends GridProcessor {
     /**
+     * @param ctx Context.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException;
+
+    /**
      * @see GridComponent#onKernalStart()
      * @throws IgniteCheckedException If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 75d65de..6630c7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -235,6 +235,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
+    @Override public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index fd798df..e96e646 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -30,12 +30,15 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import javax.cache.event.CacheEntryUpdatedListener;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
@@ -64,7 +67,6 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
@@ -83,6 +85,7 @@ import org.jsr166.ConcurrentHashMap8;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
+import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CONTINUOUS;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.processors.continuous.GridContinuousMessageType.MSG_EVT_ACK;
@@ -137,6 +140,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     /** */
     private boolean processorStopped;
 
+    /** Query sequence number for message topic. */
+    private final AtomicLong seq = new AtomicLong();
+
     /**
      * @param ctx Kernal context.
      */
@@ -255,13 +261,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                         UUID routineId = msg.routineId();
 
                         unregisterRemote(routineId);
+                    }
 
-                        if (snd.isClient()) {
-                            Map<UUID, LocalRoutineInfo> clientRoutineMap = clientInfos.get(snd.id());
-
-                            if (clientRoutineMap != null)
-                                clientRoutineMap.remove(msg.routineId());
-                        }
+                    for (Map<UUID, LocalRoutineInfo> clientInfo : clientInfos.values()) {
+                        if (clientInfo.remove(msg.routineId()) != null)
+                            break;
                     }
                 }
             });
@@ -310,6 +314,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             }
         });
 
+        ctx.marshallerContext().onContinuousProcessorStarted(ctx);
+
+        ctx.cacheObjects().onContinuousProcessorStarted(ctx);
+
+        ctx.service().onContinuousProcessorStarted(ctx);
+
         if (log.isDebugEnabled())
             log.debug("Continuous processor started.");
     }
@@ -393,16 +403,33 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override @Nullable public Serializable collectDiscoveryData(UUID nodeId) {
+        if (log.isDebugEnabled()) {
+            log.debug("collectDiscoveryData [node=" + nodeId +
+                ", loc=" + ctx.localNodeId() +
+                ", locInfos=" + locInfos +
+                ", clientInfos=" + clientInfos +
+                ']');
+        }
+
         if (!nodeId.equals(ctx.localNodeId()) || !locInfos.isEmpty()) {
             Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos0 = U.newHashMap(clientInfos.size());
 
             for (Map.Entry<UUID, Map<UUID, LocalRoutineInfo>> e : clientInfos.entrySet()) {
-                Map<UUID, LocalRoutineInfo> copy = U.newHashMap(e.getValue().size());
+                Map<UUID, LocalRoutineInfo> cp = U.newHashMap(e.getValue().size());
 
                 for (Map.Entry<UUID, LocalRoutineInfo> e0 : e.getValue().entrySet())
-                    copy.put(e0.getKey(), e0.getValue());
+                    cp.put(e0.getKey(), e0.getValue());
+
+                clientInfos0.put(e.getKey(), cp);
+            }
+
+            if (nodeId.equals(ctx.localNodeId()) && ctx.discovery().localNode().isClient()) {
+                Map<UUID, LocalRoutineInfo> infos = new HashMap<>();
 
-                clientInfos0.put(e.getKey(), copy);
+                for (Map.Entry<UUID, LocalRoutineInfo> e : locInfos.entrySet())
+                    infos.put(e.getKey(), e.getValue());
+
+                clientInfos0.put(ctx.localNodeId(), infos);
             }
 
             DiscoveryData data = new DiscoveryData(ctx.localNodeId(), clientInfos0);
@@ -430,6 +457,14 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable obj) {
         DiscoveryData data = (DiscoveryData)obj;
 
+        if (log.isDebugEnabled()) {
+            log.info("onDiscoveryDataReceived [joining=" + joiningNodeId +
+                ", rmtNodeId=" + rmtNodeId +
+                ", loc=" + ctx.localNodeId() +
+                ", data=" + data +
+                ']');
+        }
+
         if (!ctx.isDaemon() && data != null) {
             for (DiscoveryDataItem item : data.items) {
                 try {
@@ -457,29 +492,31 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             for (Map.Entry<UUID, Map<UUID, LocalRoutineInfo>> entry : data.clientInfos.entrySet()) {
                 UUID clientNodeId = entry.getKey();
 
-                Map<UUID, LocalRoutineInfo> clientRoutineMap = entry.getValue();
+                if (!ctx.localNodeId().equals(clientNodeId)) {
+                    Map<UUID, LocalRoutineInfo> clientRoutineMap = entry.getValue();
 
-                for (Map.Entry<UUID, LocalRoutineInfo> e : clientRoutineMap.entrySet()) {
-                    UUID routineId = e.getKey();
-                    LocalRoutineInfo info = e.getValue();
+                    for (Map.Entry<UUID, LocalRoutineInfo> e : clientRoutineMap.entrySet()) {
+                        UUID routineId = e.getKey();
+                        LocalRoutineInfo info = e.getValue();
 
-                    try {
-                        if (info.prjPred != null)
-                            ctx.resource().injectGeneric(info.prjPred);
-
-                        if (info.prjPred == null || info.prjPred.apply(ctx.discovery().localNode())) {
-                            if (registerHandler(clientNodeId,
-                                routineId,
-                                info.hnd,
-                                info.bufSize,
-                                info.interval,
-                                info.autoUnsubscribe,
-                                false))
-                                info.hnd.onListenerRegistered(routineId, ctx);
+                        try {
+                            if (info.prjPred != null)
+                                ctx.resource().injectGeneric(info.prjPred);
+
+                            if (info.prjPred == null || info.prjPred.apply(ctx.discovery().localNode())) {
+                                if (registerHandler(clientNodeId,
+                                    routineId,
+                                    info.hnd,
+                                    info.bufSize,
+                                    info.interval,
+                                    info.autoUnsubscribe,
+                                    false))
+                                    info.hnd.onListenerRegistered(routineId, ctx);
+                            }
+                        }
+                        catch (IgniteCheckedException err) {
+                            U.error(log, "Failed to register continuous handler.", err);
                         }
-                    }
-                    catch (IgniteCheckedException err) {
-                        U.error(log, "Failed to register continuous handler.", err);
                     }
                 }
 
@@ -537,6 +574,47 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Registers routine info to be sent in discovery data during this node join
+     * (to be used for internal queries started from client nodes).
+     *
+     * @param cacheName Cache name.
+     * @param locLsnr Local listener.
+     * @param rmtFilter Remote filter.
+     * @param prjPred Projection predicate.
+     * @return Routine ID.
+     * @throws IgniteCheckedException If failed.
+     */
+    public UUID registerStaticRoutine(
+        String cacheName,
+        CacheEntryUpdatedListener<?, ?> locLsnr,
+        CacheEntryEventSerializableFilter rmtFilter,
+        @Nullable IgnitePredicate<ClusterNode> prjPred) throws IgniteCheckedException {
+        String topicPrefix = "CONTINUOUS_QUERY_STATIC" + "_" + cacheName;
+
+        CacheContinuousQueryHandler hnd = new CacheContinuousQueryHandler(
+            cacheName,
+            TOPIC_CACHE.topic(topicPrefix, ctx.localNodeId(), seq.incrementAndGet()),
+            locLsnr,
+            rmtFilter,
+            true,
+            false,
+            true,
+            false);
+
+        hnd.internal(true);
+
+        final UUID routineId = UUID.randomUUID();
+
+        LocalRoutineInfo routineInfo = new LocalRoutineInfo(prjPred, hnd, 1, 0, true);
+
+        locInfos.put(routineId, routineInfo);
+
+        registerMessageListener(hnd);
+
+        return routineId;
+    }
+
+    /**
      * @param hnd Handler.
      * @param bufSize Buffer size.
      * @param interval Time interval.
@@ -610,29 +688,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         }
 
         // Register per-routine notifications listener if ordered messaging is used.
-        if (hnd.orderedTopic() != null) {
-            ctx.io().addMessageListener(hnd.orderedTopic(), new GridMessageListener() {
-                @Override public void onMessage(UUID nodeId, Object obj) {
-                    GridContinuousMessage msg = (GridContinuousMessage)obj;
-
-                    // Only notification can be ordered.
-                    assert msg.type() == MSG_EVT_NOTIFICATION;
-
-                    if (msg.data() == null && msg.dataBytes() != null) {
-                        try {
-                            msg.data(marsh.unmarshal(msg.dataBytes(), U.resolveClassLoader(ctx.config())));
-                        }
-                        catch (IgniteCheckedException e) {
-                            U.error(log, "Failed to process message (ignoring): " + msg, e);
-
-                            return;
-                        }
-                    }
-
-                    processNotification(nodeId, msg);
-                }
-            });
-        }
+        registerMessageListener(hnd);
 
         StartFuture fut = new StartFuture(ctx, routineId);
 
@@ -664,6 +720,35 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param hnd Handler.
+     */
+    private void registerMessageListener(GridContinuousHandler hnd) {
+        if (hnd.orderedTopic() != null) {
+            ctx.io().addMessageListener(hnd.orderedTopic(), new GridMessageListener() {
+                @Override public void onMessage(UUID nodeId, Object obj) {
+                    GridContinuousMessage msg = (GridContinuousMessage)obj;
+
+                    // Only notification can be ordered.
+                    assert msg.type() == MSG_EVT_NOTIFICATION;
+
+                    if (msg.data() == null && msg.dataBytes() != null) {
+                        try {
+                            msg.data(marsh.unmarshal(msg.dataBytes(), U.resolveClassLoader(ctx.config())));
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to process message (ignoring): " + msg, e);
+
+                            return;
+                        }
+                    }
+
+                    processNotification(nodeId, msg);
+                }
+            });
+        }
+    }
+
+    /**
      * @param routineId Consume ID.
      * @return Future.
      */
@@ -807,12 +892,28 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
         cancelFutures(new IgniteClientDisconnectedCheckedException(reconnectFut, "Client node disconnected."));
 
-        for (UUID rmtId : rmtInfos.keySet())
-            unregisterRemote(rmtId);
+        if (log.isDebugEnabled()) {
+            log.debug("onDisconnected [rmtInfos=" + rmtInfos +
+                ", locInfos=" + locInfos +
+                ", clientInfos=" + clientInfos + ']');
+        }
+
+        for (Map.Entry<UUID, RemoteRoutineInfo> e : rmtInfos.entrySet()) {
+            RemoteRoutineInfo info = e.getValue();
+
+            if (!ctx.localNodeId().equals(info.nodeId) || info.autoUnsubscribe)
+                unregisterRemote(e.getKey());
+        }
 
         rmtInfos.clear();
 
         clientInfos.clear();
+
+        if (log.isDebugEnabled()) {
+            log.debug("after onDisconnected [rmtInfos=" + rmtInfos +
+                ", locInfos=" + locInfos +
+                ", clientInfos=" + clientInfos + ']');
+        }
     }
 
     /**
@@ -1038,6 +1139,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         }
 
         if (doRegister) {
+            if (log.isDebugEnabled())
+                log.debug("Register handler: [nodeId=" + nodeId + ", routineId=" + routineId + ", info=" + info + ']');
+
             if (interval > 0) {
                 IgniteThread checker = new IgniteThread(new GridWorker(ctx.gridName(), "continuous-buffer-checker", log) {
                     @SuppressWarnings("ConstantConditions")
@@ -1152,6 +1256,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             stopLock.unlock();
         }
 
+        if (log.isDebugEnabled())
+            log.debug("unregisterRemote [routineId=" + routineId + ", loc=" + loc + ", rmt=" + remote + ']');
+
         if (remote != null)
             unregisterHandler(routineId, remote.hnd, false);
         else if (loc != null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/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 97d9988..853e6bf 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
@@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.cache.CacheIteratorConverter;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
@@ -144,12 +145,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Topology listener. */
     private GridLocalEventListener topLsnr = new TopologyListener();
 
-    /** Deployment listener ID. */
-    private UUID cfgQryId;
-
-    /** Assignment listener ID. */
-    private UUID assignQryId;
-
     /**
      * @param ctx Kernal context.
      */
@@ -166,6 +161,22 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             new ServicesCompatibilityState(srvcCompatibilitySysProp != null ? srvcCompatibilitySysProp : false, false));
     }
 
+    /**
+     * @param ctx Context.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException {
+        if (ctx.clientNode()) {
+            assert !ctx.isDaemon();
+
+            ctx.continuous().registerStaticRoutine(
+                CU.UTILITY_CACHE_NAME,
+                new ServiceEntriesListener(),
+                null,
+                null);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         ctx.addNodeAttribute(ATTR_SERVICES_COMPATIBILITY_MODE, srvcCompatibilitySysProp);
@@ -197,13 +208,32 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             if (ctx.deploy().enabled())
                 ctx.cache().context().deploy().ignoreOwnership(true);
 
-            boolean affNode = cache.context().affinityNode();
+            if (!ctx.clientNode()) {
+                assert cache.context().affinityNode();
+
+                cache.context().continuousQueries().executeInternalQuery(new ServiceEntriesListener(),
+                    null,
+                    true,
+                    true,
+                    false);
+            }
+            else {
+                assert !ctx.isDaemon();
 
-            cfgQryId = cache.context().continuousQueries().executeInternalQuery(
-                new DeploymentListener(), null, affNode, true, !affNode);
+                ctx.closure().runLocalSafe(new Runnable() {
+                    @Override public void run() {
+                        try {
+                            Iterable<CacheEntryEvent<?, ?>> entries =
+                                cache.context().continuousQueries().existingEntries(false, null);
 
-            assignQryId = cache.context().continuousQueries().executeInternalQuery(
-                new AssignmentListener(), null, affNode, true, !affNode);
+                            onSystemCacheUpdated(entries);
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to load service entries: " + e, e);
+                        }
+                    }
+                });
+            }
         }
         finally {
             if (ctx.deploy().enabled())
@@ -237,12 +267,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         if (!ctx.clientNode())
             ctx.event().removeLocalEventListener(topLsnr);
 
-        if (cfgQryId != null)
-            cache.context().continuousQueries().cancelInternalQuery(cfgQryId);
-
-        if (assignQryId != null)
-            cache.context().continuousQueries().cancelInternalQuery(assignQryId);
-
         Collection<ServiceContextImpl> ctxs = new ArrayList<>();
 
         synchronized (locSvcs) {
@@ -1284,148 +1308,171 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /**
      * Service deployment listener.
      */
-    private class DeploymentListener implements CacheEntryUpdatedListener<Object, Object> {
+    @SuppressWarnings("unchecked")
+    private class ServiceEntriesListener implements CacheEntryUpdatedListener<Object, Object> {
         /** {@inheritDoc} */
         @Override public void onUpdated(final Iterable<CacheEntryEvent<?, ?>> deps) {
             depExe.submit(new BusyRunnable() {
                 @Override public void run0() {
-                    boolean firstTime = true;
+                    onSystemCacheUpdated(deps);
+                }
+            });
+        }
+    }
 
-                    for (CacheEntryEvent<?, ?> e : deps) {
-                        if (!(e.getKey() instanceof GridServiceDeploymentKey))
-                            continue;
+    /**
+     * @param evts Update events.
+     */
+    private void onSystemCacheUpdated(final Iterable<CacheEntryEvent<?, ?>> evts) {
+        boolean firstTime = true;
 
-                        if (firstTime) {
-                            markCompatibilityStateAsUsed();
+        for (CacheEntryEvent<?, ?> e : evts) {
+            if (e.getKey() instanceof GridServiceDeploymentKey) {
+                if (firstTime) {
+                    markCompatibilityStateAsUsed();
 
-                            firstTime = false;
-                        }
+                    firstTime = false;
+                }
 
-                        GridServiceDeployment dep;
+                processDeployment((CacheEntryEvent)e);
+            }
+            else if (e.getKey() instanceof GridServiceAssignmentsKey) {
+                if (firstTime) {
+                    markCompatibilityStateAsUsed();
 
-                        try {
-                            dep = (GridServiceDeployment)e.getValue();
-                        }
-                        catch (IgniteException ex) {
-                            if (X.hasCause(ex, ClassNotFoundException.class))
-                                continue;
-                            else
-                                throw ex;
-                        }
+                    firstTime = false;
+                }
+
+                processAssignment((CacheEntryEvent)e);
+            }
+        }
+    }
 
-                        if (dep != null) {
-                            svcName.set(dep.configuration().getName());
+    /**
+     * @param e Entry.
+     */
+    private void processDeployment(CacheEntryEvent<GridServiceDeploymentKey, GridServiceDeployment> e) {
+        GridServiceDeployment dep;
 
-                            // Ignore other utility cache events.
-                            long topVer = ctx.discovery().topologyVersion();
+        try {
+            dep = e.getValue();
+        }
+        catch (IgniteException ex) {
+            if (X.hasCause(ex, ClassNotFoundException.class))
+                return;
+            else
+                throw ex;
+        }
 
-                            ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
+        if (dep != null) {
+            svcName.set(dep.configuration().getName());
 
-                            if (oldest.isLocal())
-                                onDeployment(dep, topVer);
-                        }
-                        // Handle undeployment.
-                        else {
-                            String name = ((GridServiceDeploymentKey)e.getKey()).name();
+            // Ignore other utility cache events.
+            long topVer = ctx.discovery().topologyVersion();
 
-                            svcName.set(name);
+            ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
 
-                            Collection<ServiceContextImpl> ctxs;
+            if (oldest.isLocal())
+                onDeployment(dep, topVer);
+        }
+        // Handle undeployment.
+        else {
+            String name = e.getKey().name();
 
-                            synchronized (locSvcs) {
-                                ctxs = locSvcs.remove(name);
-                            }
+            svcName.set(name);
 
-                            if (ctxs != null) {
-                                synchronized (ctxs) {
-                                    cancel(ctxs, ctxs.size());
-                                }
-                            }
+            Collection<ServiceContextImpl> ctxs;
 
-                            // Finish deployment futures if undeployment happened.
-                            GridFutureAdapter<?> fut = depFuts.remove(name);
+            synchronized (locSvcs) {
+                ctxs = locSvcs.remove(name);
+            }
 
-                            if (fut != null)
-                                fut.onDone();
+            if (ctxs != null) {
+                synchronized (ctxs) {
+                    cancel(ctxs, ctxs.size());
+                }
+            }
 
-                            // Complete undeployment future.
-                            fut = undepFuts.remove(name);
+            // Finish deployment futures if undeployment happened.
+            GridFutureAdapter<?> fut = depFuts.remove(name);
 
-                            if (fut != null)
-                                fut.onDone();
+            if (fut != null)
+                fut.onDone();
 
-                            GridServiceAssignmentsKey key = new GridServiceAssignmentsKey(name);
+            // Complete undeployment future.
+            fut = undepFuts.remove(name);
 
-                            // Remove assignment on primary node in case of undeploy.
-                            if (cache.cache().affinity().isPrimary(ctx.discovery().localNode(), key)) {
-                                try {
-                                    cache.getAndRemove(key);
-                                }
-                                catch (IgniteCheckedException ex) {
-                                    U.error(log, "Failed to remove assignments for undeployed service: " + name, ex);
-                                }
-                            }
-                        }
-                    }
+            if (fut != null)
+                fut.onDone();
+
+            GridServiceAssignmentsKey key = new GridServiceAssignmentsKey(name);
+
+            // Remove assignment on primary node in case of undeploy.
+            if (cache.cache().affinity().isPrimary(ctx.discovery().localNode(), key)) {
+                try {
+                    cache.getAndRemove(key);
                 }
-            });
+                catch (IgniteCheckedException ex) {
+                    U.error(log, "Failed to remove assignments for undeployed service: " + name, ex);
+                }
+            }
         }
+    }
 
-        /**
-         * Deployment callback.
-         *
-         * @param dep Service deployment.
-         * @param topVer Topology version.
-         */
-        private void onDeployment(final GridServiceDeployment dep, final long topVer) {
-            // Retry forever.
-            try {
-                long newTopVer = ctx.discovery().topologyVersion();
+    /**
+     * Deployment callback.
+     *
+     * @param dep Service deployment.
+     * @param topVer Topology version.
+     */
+    private void onDeployment(final GridServiceDeployment dep, final long topVer) {
+        // Retry forever.
+        try {
+            long newTopVer = ctx.discovery().topologyVersion();
 
-                // If topology version changed, reassignment will happen from topology event.
-                if (newTopVer == topVer)
-                    reassign(dep, topVer);
-            }
-            catch (IgniteCheckedException e) {
-                if (!(e instanceof ClusterTopologyCheckedException))
-                    log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
+            // If topology version changed, reassignment will happen from topology event.
+            if (newTopVer == topVer)
+                reassign(dep, topVer);
+        }
+        catch (IgniteCheckedException e) {
+            if (!(e instanceof ClusterTopologyCheckedException))
+                log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
 
-                long newTopVer = ctx.discovery().topologyVersion();
+            long newTopVer = ctx.discovery().topologyVersion();
 
-                if (newTopVer != topVer) {
-                    assert newTopVer > topVer;
+            if (newTopVer != topVer) {
+                assert newTopVer > topVer;
 
-                    // Reassignment will happen from topology event.
-                    return;
-                }
+                // Reassignment will happen from topology event.
+                return;
+            }
 
-                ctx.timeout().addTimeoutObject(new GridTimeoutObject() {
-                    private IgniteUuid id = IgniteUuid.randomUuid();
+            ctx.timeout().addTimeoutObject(new GridTimeoutObject() {
+                private IgniteUuid id = IgniteUuid.randomUuid();
 
-                    private long start = System.currentTimeMillis();
+                private long start = System.currentTimeMillis();
 
-                    @Override public IgniteUuid timeoutId() {
-                        return id;
-                    }
+                @Override public IgniteUuid timeoutId() {
+                    return id;
+                }
 
-                    @Override public long endTime() {
-                        return start + RETRY_TIMEOUT;
-                    }
+                @Override public long endTime() {
+                    return start + RETRY_TIMEOUT;
+                }
 
-                    @Override public void onTimeout() {
-                        if (!busyLock.enterBusy())
-                            return;
+                @Override public void onTimeout() {
+                    if (!busyLock.enterBusy())
+                        return;
 
-                        try {
-                            // Try again.
-                            onDeployment(dep, topVer);
-                        }
-                        finally {
-                            busyLock.leaveBusy();
-                        }
+                    try {
+                        // Try again.
+                        onDeployment(dep, topVer);
                     }
-                });
-            }
+                    finally {
+                        busyLock.leaveBusy();
+                    }
+                }
+            });
         }
     }
 
@@ -1585,79 +1632,59 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Assignment listener.
+     * @param e Entry.
      */
-    private class AssignmentListener implements CacheEntryUpdatedListener<Object, Object> {
-        /** {@inheritDoc} */
-        @Override public void onUpdated(final Iterable<CacheEntryEvent<?, ?>> assignCol) throws CacheEntryListenerException {
-            depExe.submit(new BusyRunnable() {
-                @Override public void run0() {
-                    boolean firstTime = true;
-
-                    for (CacheEntryEvent<?, ?> e : assignCol) {
-                        if (!(e.getKey() instanceof GridServiceAssignmentsKey))
-                            continue;
-
-                        if (firstTime) {
-                            markCompatibilityStateAsUsed();
+    private void processAssignment(CacheEntryEvent<GridServiceAssignmentsKey, GridServiceAssignments> e) {
+        GridServiceAssignments assigns;
 
-                            firstTime = false;
-                        }
-
-                        GridServiceAssignments assigns;
-
-                        try {
-                            assigns = (GridServiceAssignments)e.getValue();
-                        }
-                        catch (IgniteException ex) {
-                            if (X.hasCause(ex, ClassNotFoundException.class))
-                                continue;
-                            else
-                                throw ex;
-                        }
+        try {
+            assigns = e.getValue();
+        }
+        catch (IgniteException ex) {
+            if (X.hasCause(ex, ClassNotFoundException.class))
+                return;
+            else
+                throw ex;
+        }
 
-                        if (assigns != null) {
-                            svcName.set(assigns.name());
+        if (assigns != null) {
+            svcName.set(assigns.name());
 
-                            Throwable t = null;
+            Throwable t = null;
 
-                            try {
-                                redeploy(assigns);
-                            }
-                            catch (Error | RuntimeException th) {
-                                t = th;
-                            }
+            try {
+                redeploy(assigns);
+            }
+            catch (Error | RuntimeException th) {
+                t = th;
+            }
 
-                            GridServiceDeploymentFuture fut = depFuts.get(assigns.name());
+            GridServiceDeploymentFuture fut = depFuts.get(assigns.name());
 
-                            if (fut != null && fut.configuration().equalsIgnoreNodeFilter(assigns.configuration())) {
-                                depFuts.remove(assigns.name(), fut);
+            if (fut != null && fut.configuration().equalsIgnoreNodeFilter(assigns.configuration())) {
+                depFuts.remove(assigns.name(), fut);
 
-                                // Complete deployment futures once the assignments have been stored in cache.
-                                fut.onDone(null, t);
-                            }
-                        }
-                        // Handle undeployment.
-                        else {
-                            String name = ((GridServiceAssignmentsKey)e.getKey()).name();
+                // Complete deployment futures once the assignments have been stored in cache.
+                fut.onDone(null, t);
+            }
+        }
+        // Handle undeployment.
+        else {
+            String name = e.getKey().name();
 
-                            svcName.set(name);
+            svcName.set(name);
 
-                            Collection<ServiceContextImpl> ctxs;
+            Collection<ServiceContextImpl> ctxs;
 
-                            synchronized (locSvcs) {
-                                ctxs = locSvcs.remove(name);
-                            }
+            synchronized (locSvcs) {
+                ctxs = locSvcs.remove(name);
+            }
 
-                            if (ctxs != null) {
-                                synchronized (ctxs) {
-                                    cancel(ctxs, ctxs.size());
-                                }
-                            }
-                        }
-                    }
+            if (ctxs != null) {
+                synchronized (ctxs) {
+                    cancel(ctxs, ctxs.size());
                 }
-            });
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
index 4c44adc..0ff5883 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
@@ -125,6 +125,7 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
     }
 
     /**
+     * @param stopFromClient If {@code true} stops listener from client node, otherwise from server.
      * @throws Exception If failed.
      */
     private void testMessageListenerReconnect(boolean stopFromClient) throws Exception {
@@ -178,9 +179,11 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
         assertTrue(locLsnr.latch.await(5000, MILLISECONDS));
         assertTrue(latch.await(5000, MILLISECONDS));
 
-        log.info("Stop listen, should not get remote messages anymore.");
+        Ignite stopFrom = (stopFromClient ? client : srv);
 
-        (stopFromClient ? client : srv).message().stopRemoteListen(opId);
+        log.info("Stop listen, should not get remote messages anymore [from=" + stopFrom.name() + ']');
+
+        stopFrom.message().stopRemoteListen(opId);
 
         srv.message().send(topic, "msg3");
 
@@ -243,6 +246,59 @@ public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientRe
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testCacheContinuousQueryReconnectNewServer() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>());
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setAutoUnsubscribe(true);
+
+        qry.setLocalListener(lsnr);
+
+        QueryCursor<?> cur = clientCache.query(qry);
+
+        continuousQueryReconnect(client, clientCache, lsnr);
+
+        // Check new server registers listener for reconnected client.
+        try (Ignite newSrv = startGrid(serverCount() + 1)) {
+            awaitPartitionMapExchange();
+
+            lsnr.latch = new CountDownLatch(10);
+
+            IgniteCache<Object, Object> newSrvCache = newSrv.cache(null);
+
+            for (Integer key : primaryKeys(newSrvCache, 10))
+                newSrvCache.put(key, key);
+
+            assertTrue(lsnr.latch.await(5000, MILLISECONDS));
+        }
+
+        cur.close();
+
+        // Check new server does not register listener for closed query.
+        try (Ignite newSrv = startGrid(serverCount() + 1)) {
+            awaitPartitionMapExchange();
+
+            lsnr.latch = new CountDownLatch(5);
+
+            IgniteCache<Object, Object> newSrvCache = newSrv.cache(null);
+
+            for (Integer key : primaryKeys(newSrvCache, 5))
+                newSrvCache.put(key, key);
+
+            assertFalse(lsnr.latch.await(3000, MILLISECONDS));
+        }
+    }
+
+    /**
      * @param client Client.
      * @param clientCache Client cache.
      * @param lsnr Continuous query listener.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 3ae6bb4..3d238af 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -209,7 +209,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         for (int i = 0; i < gridCount(); i++) {
             GridContinuousProcessor proc = grid(i).context().continuous();
 
-            assertEquals(String.valueOf(i), 3, ((Map)U.field(proc, "locInfos")).size());
+            assertEquals(String.valueOf(i), 2, ((Map)U.field(proc, "locInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "rmtInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "startFuts")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "stopFuts")).size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
new file mode 100644
index 0000000..05537c0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/IgniteNoCustomEventsOnNodeStart.java
@@ -0,0 +1,80 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+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.junits.common.GridCommonAbstractTest;
+
+/**
+ * Sanity test to verify there are no unnecessary messages on node start.
+ */
+public class IgniteNoCustomEventsOnNodeStart extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private static volatile boolean failed;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TestTcpDiscoverySpi discoSpi = new TestTcpDiscoverySpi();
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoCustomEventsOnStart() throws Exception {
+        failed = false;
+
+        for (int i = 0; i < 5; i++) {
+            client = i % 2 == 1;
+
+            startGrid(i);
+        }
+
+        assertFalse(failed);
+    }
+
+    /**
+     *
+     */
+    static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+        /** {@inheritDoc} */
+        @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) {
+            failed = true;
+
+            fail("Should not be called.");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
index c3b9cf4..665294d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
@@ -21,6 +21,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.IgniteConfiguration;
+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;
@@ -34,47 +35,118 @@ public class GridServiceClientNodeTest extends GridCommonAbstractTest {
     protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** */
-    private static final int NODE_CNT = 3;
+    private boolean client;
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi) cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setMaxMissedClientHeartbeats(30);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setHeartbeatFrequency(1000);
 
-        if (gridName.equals(getTestGridName(NODE_CNT - 1)))
-            cfg.setClientMode(true);
+        cfg.setClientMode(client);
 
         return cfg;
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
 
-        startGrids(NODE_CNT);
+        super.afterTest();
     }
 
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployFromClient() throws Exception {
+        startGrids(3);
+
+        client = true;
+
+        Ignite ignite = startGrid(3);
+
+        checkDeploy(ignite, "service1");
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testDeployFromClient() throws Exception {
-        Ignite ignite = ignite(NODE_CNT - 1);
+    public void testDeployFromClientAfterRouterStop1() throws Exception {
+        startGrid(0);
 
-        assertTrue(ignite.configuration().isClientMode());
+        client = true;
 
-        String svcName = "testService";
+        Ignite ignite = startGrid(1);
+
+        client = false;
+
+        startGrid(2);
+
+        U.sleep(1000);
+
+        stopGrid(0);
+
+        awaitPartitionMapExchange();
+
+        checkDeploy(ignite, "service1");
+
+        startGrid(3);
+
+        for (int i = 0; i < 10; i++)
+            checkDeploy(ignite, "service2-" + i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployFromClientAfterRouterStop2() throws Exception {
+        startGrid(0);
+
+        client = true;
+
+        Ignite ignite = startGrid(1);
+
+        client = false;
+
+        startGrid(2);
+
+        client = true;
+
+        startGrid(3);
+
+        client = false;
+
+        startGrid(4);
+
+        U.sleep(1000);
+
+        stopGrid(0);
+
+        awaitPartitionMapExchange();
+
+        checkDeploy(ignite, "service1");
+
+        startGrid(5);
+
+        for (int i = 0; i < 10; i++)
+            checkDeploy(ignite, "service2-" + i);
+    }
+
+    /**
+     * @param client Client node.
+     * @param svcName Service name.
+     * @throws Exception If failed.
+     */
+    private void checkDeploy(Ignite client, String svcName) throws Exception {
+        assertTrue(client.configuration().isClientMode());
 
         CountDownLatch latch = new CountDownLatch(1);
 
         DummyService.exeLatch(svcName, latch);
 
-        ignite.services().deployClusterSingleton(svcName, new DummyService());
+        client.services().deployClusterSingleton(svcName, new DummyService());
 
         assertTrue(latch.await(5000, TimeUnit.MILLISECONDS));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7f878c56/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 d0943b5..2632d4c 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
@@ -125,6 +125,7 @@ import org.apache.ignite.internal.processors.cache.local.GridCacheLocalMultithre
 import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxSingleThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.local.GridCacheLocalTxTimeoutSelfTest;
+import org.apache.ignite.internal.processors.continuous.IgniteNoCustomEventsOnNodeStart;
 
 /**
  * Test suite.
@@ -253,6 +254,8 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
         suite.addTest(new TestSuite(CacheEnumOperationsTest.class));
 
+        suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class));
+
         return suite;
     }
 }


[12/50] ignite git commit: Merge remote-tracking branch 'remotes/community/gridgain-7.5.26' into ignite-3209

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/community/gridgain-7.5.26' into ignite-3209


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

Branch: refs/heads/ignite-3341
Commit: e7b85c28d5d836f62182d1c3511ccad8cb03cd08
Parents: 1d80a39 5f44672
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 15 10:14:44 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 15 10:14:44 2016 +0300

----------------------------------------------------------------------
 .../configuration/FileSystemConfiguration.java  |  18 +-
 .../continuous/CacheContinuousQueryEntry.java   |  16 ++
 .../continuous/CacheContinuousQueryHandler.java |   2 +-
 .../processors/igfs/IgfsDataManager.java        | 166 ++++---------------
 .../igfs/data/IgfsDataPutProcessor.java         |  99 +++++++++++
 ...niteCacheContinuousQueryBackupQueueTest.java | 135 +++++++++++++++
 .../IgniteCacheQuerySelfTestSuite3.java         |   2 +
 7 files changed, 301 insertions(+), 137 deletions(-)
----------------------------------------------------------------------



[32/50] ignite git commit: Merge branch 'gridgain-7.5.26' of https://github.com/gridgain/apache-ignite into gridgain-7.5.26

Posted by vo...@apache.org.
Merge branch 'gridgain-7.5.26' of https://github.com/gridgain/apache-ignite into gridgain-7.5.26


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

Branch: refs/heads/ignite-3341
Commit: fb23e00df7a693088acc2b7d29564fc0e29ba3be
Parents: 377b387 2c18b6e
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jun 15 17:18:39 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jun 15 17:18:39 2016 +0300

----------------------------------------------------------------------
 modules/cassandra/README.txt                    |  32 +
 modules/cassandra/licenses/apache-2.0.txt       | 202 +++++
 modules/cassandra/pom.xml                       | 340 +++++++
 .../store/cassandra/CassandraCacheStore.java    | 409 +++++++++
 .../cassandra/CassandraCacheStoreFactory.java   | 200 +++++
 .../store/cassandra/common/CassandraHelper.java | 133 +++
 .../cassandra/common/PropertyMappingHelper.java | 220 +++++
 .../store/cassandra/common/RandomSleeper.java   | 104 +++
 .../store/cassandra/common/SystemHelper.java    |  46 +
 .../store/cassandra/datasource/Credentials.java |  37 +
 .../store/cassandra/datasource/DataSource.java  | 550 ++++++++++++
 .../cassandra/datasource/PlainCredentials.java  |  50 ++
 .../persistence/KeyPersistenceSettings.java     | 274 ++++++
 .../KeyValuePersistenceSettings.java            | 478 ++++++++++
 .../persistence/PersistenceController.java      | 421 +++++++++
 .../persistence/PersistenceSettings.java        | 335 +++++++
 .../persistence/PersistenceStrategy.java        |  62 ++
 .../store/cassandra/persistence/PojoField.java  | 219 +++++
 .../cassandra/persistence/PojoKeyField.java     |  91 ++
 .../cassandra/persistence/PojoValueField.java   | 152 ++++
 .../persistence/ValuePersistenceSettings.java   | 107 +++
 .../cassandra/serializer/JavaSerializer.java    |  81 ++
 .../cassandra/serializer/KryoSerializer.java    |  93 ++
 .../store/cassandra/serializer/Serializer.java  |  43 +
 .../session/BatchExecutionAssistant.java        |  95 ++
 .../cassandra/session/BatchLoaderAssistant.java |  47 +
 .../cassandra/session/CassandraSession.java     |  60 ++
 .../cassandra/session/CassandraSessionImpl.java | 832 +++++++++++++++++
 .../cassandra/session/ExecutionAssistant.java   |  77 ++
 .../session/GenericBatchExecutionAssistant.java |  67 ++
 .../session/LoadCacheCustomQueryWorker.java     | 105 +++
 .../cassandra/session/pool/SessionPool.java     | 173 ++++
 .../cassandra/session/pool/SessionWrapper.java  |  71 ++
 .../store/cassandra/utils/DDLGenerator.java     |  64 ++
 .../aws/cassandra/cassandra-bootstrap.sh        | 373 ++++++++
 .../bootstrap/aws/cassandra/cassandra-env.sh    | 283 ++++++
 .../bootstrap/aws/cassandra/cassandra-start.sh  | 550 ++++++++++++
 .../aws/cassandra/cassandra-template.yaml       | 889 +++++++++++++++++++
 .../bootstrap/aws/ignite/ignite-bootstrap.sh    | 384 ++++++++
 .../ignite/ignite-cassandra-server-template.xml | 177 ++++
 .../src/test/bootstrap/aws/ignite/ignite-env.sh |  25 +
 .../test/bootstrap/aws/ignite/ignite-start.sh   | 637 +++++++++++++
 .../src/test/bootstrap/aws/logs-collector.sh    | 102 +++
 .../tests/ignite-cassandra-client-template.xml  | 173 ++++
 .../test/bootstrap/aws/tests/tests-bootstrap.sh | 379 ++++++++
 .../test/bootstrap/aws/tests/tests-report.sh    | 590 ++++++++++++
 .../src/test/bootstrap/aws/tests/tests-run.sh   | 715 +++++++++++++++
 .../CassandraDirectPersistenceLoadTest.java     | 107 +++
 .../tests/CassandraDirectPersistenceTest.java   | 371 ++++++++
 .../apache/ignite/tests/DDLGeneratorTest.java   |  43 +
 .../tests/IgnitePersistentStoreLoadTest.java    | 111 +++
 .../ignite/tests/IgnitePersistentStoreTest.java | 369 ++++++++
 .../org/apache/ignite/tests/load/Generator.java |  27 +
 .../apache/ignite/tests/load/IntGenerator.java  |  33 +
 .../ignite/tests/load/LoadTestDriver.java       | 238 +++++
 .../apache/ignite/tests/load/LongGenerator.java |  28 +
 .../ignite/tests/load/PersonGenerator.java      |  43 +
 .../ignite/tests/load/PersonIdGenerator.java    |  31 +
 .../ignite/tests/load/StringGenerator.java      |  28 +
 .../org/apache/ignite/tests/load/Worker.java    | 429 +++++++++
 .../tests/load/cassandra/BulkReadWorker.java    |  63 ++
 .../tests/load/cassandra/BulkWriteWorker.java   |  52 ++
 .../ignite/tests/load/cassandra/ReadWorker.java |  51 ++
 .../tests/load/cassandra/WriteWorker.java       |  51 ++
 .../tests/load/ignite/BulkReadWorker.java       |  52 ++
 .../tests/load/ignite/BulkWriteWorker.java      |  52 ++
 .../ignite/tests/load/ignite/ReadWorker.java    |  51 ++
 .../ignite/tests/load/ignite/WriteWorker.java   |  51 ++
 .../org/apache/ignite/tests/pojos/Person.java   | 246 +++++
 .../org/apache/ignite/tests/pojos/PersonId.java | 110 +++
 .../ignite/tests/utils/CacheStoreHelper.java    |  64 ++
 .../tests/utils/CassandraAdminCredentials.java  |  36 +
 .../ignite/tests/utils/CassandraHelper.java     | 358 ++++++++
 .../tests/utils/CassandraLifeCycleBean.java     | 149 ++++
 .../utils/CassandraRegularCredentials.java      |  36 +
 .../ignite/tests/utils/TestCacheSession.java    |  91 ++
 .../apache/ignite/tests/utils/TestsHelper.java  | 375 ++++++++
 .../src/test/resources/log4j.properties         | 119 +++
 .../tests/cassandra/connection-settings.xml     |  48 +
 .../tests/cassandra/connection.properties       |  17 +
 .../tests/cassandra/credentials.properties      |  22 +
 .../tests/cassandra/embedded-cassandra.yaml     | 119 +++
 .../ignite/tests/cassandra/keyspaces.properties |  17 +
 .../tests/persistence/blob/ignite-config.xml    | 100 +++
 .../persistence/blob/persistence-settings-1.xml |  21 +
 .../persistence/blob/persistence-settings-2.xml |  21 +
 .../persistence/blob/persistence-settings-3.xml |  29 +
 .../tests/persistence/pojo/ignite-config.xml    | 119 +++
 .../persistence/pojo/persistence-settings-1.xml |  21 +
 .../persistence/pojo/persistence-settings-2.xml |  21 +
 .../persistence/pojo/persistence-settings-3.xml | 173 ++++
 .../persistence/primitive/ignite-config.xml     | 100 +++
 .../primitive/ignite-remote-client-config.xml   |  95 ++
 .../primitive/ignite-remote-server-config.xml   | 106 +++
 .../primitive/persistence-settings-1.xml        |  21 +
 .../primitive/persistence-settings-2.xml        |  21 +
 .../src/test/resources/tests.properties         |  50 ++
 .../src/test/scripts/cassandra-load-tests.bat   |  41 +
 .../src/test/scripts/cassandra-load-tests.sh    |  39 +
 .../src/test/scripts/ignite-load-tests.bat      |  41 +
 .../src/test/scripts/ignite-load-tests.sh       |  39 +
 modules/cassandra/src/test/scripts/jvm-opt.sh   |  21 +
 modules/cassandra/src/test/scripts/jvm-opts.bat |  24 +
 .../configuration/FileSystemConfiguration.java  |  18 +-
 .../ignite/internal/MarshallerContextImpl.java  |  51 +-
 .../ignite/internal/binary/BinaryContext.java   |  43 +-
 .../processors/cache/GridCacheAdapter.java      |  46 +-
 .../processors/cache/GridCacheEntryEx.java      |   9 +-
 .../processors/cache/GridCacheMapEntry.java     | 100 +--
 .../processors/cache/GridCacheUtils.java        |   3 +
 .../binary/CacheObjectBinaryProcessorImpl.java  |  29 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  10 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |  19 +-
 .../dht/GridPartitionedGetFuture.java           |   2 -
 .../dht/GridPartitionedSingleGetFuture.java     |   2 -
 .../dht/atomic/GridDhtAtomicCache.java          |   8 -
 .../dht/colocated/GridDhtColocatedCache.java    |   4 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   3 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   3 +-
 .../distributed/near/GridNearGetFuture.java     |   4 -
 .../cache/distributed/near/GridNearTxLocal.java |   7 +-
 .../local/atomic/GridLocalAtomicCache.java      |   8 -
 .../continuous/CacheContinuousQueryEntry.java   |  16 +
 .../continuous/CacheContinuousQueryHandler.java |   2 +-
 .../continuous/CacheContinuousQueryManager.java | 135 +++
 .../cache/transactions/IgniteTxAdapter.java     |   2 -
 .../cache/transactions/IgniteTxEntry.java       |  24 +-
 .../cache/transactions/IgniteTxHandler.java     |  46 +
 .../transactions/IgniteTxLocalAdapter.java      |  34 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   7 +
 .../IgniteCacheObjectProcessorImpl.java         |   5 +
 .../continuous/GridContinuousProcessor.java     | 217 +++--
 .../datastreamer/DataStreamerImpl.java          |   3 +-
 .../processors/igfs/IgfsCreateResult.java       |  66 ++
 .../processors/igfs/IgfsDataManager.java        | 166 +---
 .../internal/processors/igfs/IgfsImpl.java      |  53 +-
 .../processors/igfs/IgfsMetaManager.java        | 633 ++++++-------
 .../IgfsSecondaryFileSystemCreateContext.java   | 111 +++
 .../IgfsSecondaryOutputStreamDescriptor.java    |  59 --
 .../igfs/data/IgfsDataPutProcessor.java         |  99 +++
 .../meta/IgfsMetaDirectoryCreateProcessor.java  |  40 +-
 ...IgfsMetaDirectoryListingRenameProcessor.java | 133 +++
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  |  46 +-
 .../service/GridServiceProcessor.java           | 391 ++++----
 .../processors/task/GridTaskWorker.java         |  44 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   4 +-
 ...eClientReconnectContinuousProcessorTest.java |  60 +-
 .../ignite/internal/binary/AffinityKey.java     |  69 ++
 .../binary/GridBinaryAffinityKeySelfTest.java   |  15 +
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |  17 +
 .../cache/CacheAffinityCallSelfTest.java        |  45 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   5 +-
 .../cache/IgniteCacheAbstractTest.java          |   2 +-
 ...gniteCacheInvokeReadThroughAbstractTest.java | 382 ++++++++
 ...iteCacheInvokeReadThroughSingleNodeTest.java | 106 +++
 .../cache/IgniteCacheInvokeReadThroughTest.java | 182 ++--
 .../IgniteCacheReadThroughStoreCallTest.java    | 288 ++++++
 .../IgniteCacheLoaderWriterAbstractTest.java    |  10 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |   2 +-
 ...niteCacheContinuousQueryBackupQueueTest.java | 135 +++
 .../IgniteNoCustomEventsOnNodeStart.java        |  80 ++
 .../igfs/IgfsMetaManagerSelfTest.java           |   4 +-
 .../service/GridServiceClientNodeTest.java      | 102 ++-
 .../testsuites/IgniteCacheTestSuite2.java       |   3 +
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 .../processors/query/h2/sql/GridSqlConst.java   |   5 +
 .../processors/query/h2/sql/GridSqlJoin.java    |  17 +-
 .../processors/query/h2/sql/GridSqlType.java    |   5 +
 .../query/IgniteSqlSplitterSelfTest.java        |  75 ++
 .../IgniteCacheQuerySelfTestSuite3.java         |   2 +
 .../org/apache/ignite/spark/IgniteContext.scala |   4 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |  25 +-
 .../apache/ignite/spark/JavaIgniteContext.scala |   4 +-
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |   2 +
 .../ignite/spark/impl/IgniteAbstractRDD.scala   |  15 +-
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |   5 +-
 .../spark/impl/JavaIgniteAbstractRDD.scala      |  34 -
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |  25 +-
 pom.xml                                         |   9 +
 181 files changed, 20033 insertions(+), 1168 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fb23e00d/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
----------------------------------------------------------------------


[17/50] ignite git commit: Fixed compilation

Posted by vo...@apache.org.
Fixed compilation


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

Branch: refs/heads/ignite-3341
Commit: 79ee31f38f03d08dd684f123eaeacdc77038bd47
Parents: e235298
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Jun 15 13:44:23 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Jun 15 13:44:23 2016 +0300

----------------------------------------------------------------------
 .../spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala    | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/79ee31f3/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index fb15ff1..f198f0c 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -32,7 +32,6 @@ import org.apache.spark.sql._
 import org.apache.spark._
 
 import scala.collection.JavaConversions._
-import scala.reflect.macros.whitebox
 
 /**
  * Ignite RDD. Represents Ignite cache as Spark RDD abstraction.


[26/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
IGNITE-1371 Implemented Cassandra cache store.


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

Branch: refs/heads/ignite-3341
Commit: 83c26a918599fc2e411d653396a5669803e32d40
Parents: 79ee31f
Author: Igor <ir...@gmail.com>
Authored: Wed May 18 12:49:46 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Jun 15 15:22:15 2016 +0300

----------------------------------------------------------------------
 modules/cassandra/README.txt                    |  32 +
 modules/cassandra/licenses/apache-2.0.txt       | 202 +++++
 modules/cassandra/pom.xml                       | 340 +++++++
 .../store/cassandra/CassandraCacheStore.java    | 409 +++++++++
 .../cassandra/CassandraCacheStoreFactory.java   | 200 +++++
 .../cassandra/bean/CassandraLifeCycleBean.java  | 149 ++++
 .../store/cassandra/common/CassandraHelper.java | 133 +++
 .../cassandra/common/PropertyMappingHelper.java | 220 +++++
 .../store/cassandra/common/RandomSleeper.java   | 104 +++
 .../store/cassandra/common/SystemHelper.java    |  46 +
 .../store/cassandra/datasource/Credentials.java |  37 +
 .../store/cassandra/datasource/DataSource.java  | 550 ++++++++++++
 .../cassandra/datasource/PlainCredentials.java  |  50 ++
 .../persistence/KeyPersistenceSettings.java     | 274 ++++++
 .../KeyValuePersistenceSettings.java            | 478 ++++++++++
 .../persistence/PersistenceController.java      | 421 +++++++++
 .../persistence/PersistenceSettings.java        | 335 +++++++
 .../persistence/PersistenceStrategy.java        |  62 ++
 .../store/cassandra/persistence/PojoField.java  | 219 +++++
 .../cassandra/persistence/PojoKeyField.java     |  91 ++
 .../cassandra/persistence/PojoValueField.java   | 152 ++++
 .../persistence/ValuePersistenceSettings.java   | 107 +++
 .../cassandra/serializer/JavaSerializer.java    |  81 ++
 .../cassandra/serializer/KryoSerializer.java    |  93 ++
 .../store/cassandra/serializer/Serializer.java  |  43 +
 .../session/BatchExecutionAssistant.java        |  95 ++
 .../cassandra/session/BatchLoaderAssistant.java |  47 +
 .../cassandra/session/CassandraSession.java     |  60 ++
 .../cassandra/session/CassandraSessionImpl.java | 832 +++++++++++++++++
 .../cassandra/session/ExecutionAssistant.java   |  77 ++
 .../session/GenericBatchExecutionAssistant.java |  67 ++
 .../session/LoadCacheCustomQueryWorker.java     | 105 +++
 .../cassandra/session/pool/SessionPool.java     | 173 ++++
 .../cassandra/session/pool/SessionWrapper.java  |  71 ++
 .../store/cassandra/utils/DDLGenerator.java     |  64 ++
 .../aws/cassandra/cassandra-bootstrap.sh        | 373 ++++++++
 .../bootstrap/aws/cassandra/cassandra-env.sh    | 283 ++++++
 .../bootstrap/aws/cassandra/cassandra-start.sh  | 550 ++++++++++++
 .../aws/cassandra/cassandra-template.yaml       | 889 +++++++++++++++++++
 .../bootstrap/aws/ignite/ignite-bootstrap.sh    | 384 ++++++++
 .../ignite/ignite-cassandra-server-template.xml | 177 ++++
 .../src/test/bootstrap/aws/ignite/ignite-env.sh |  25 +
 .../test/bootstrap/aws/ignite/ignite-start.sh   | 637 +++++++++++++
 .../src/test/bootstrap/aws/logs-collector.sh    | 102 +++
 .../tests/ignite-cassandra-client-template.xml  | 173 ++++
 .../test/bootstrap/aws/tests/tests-bootstrap.sh | 379 ++++++++
 .../test/bootstrap/aws/tests/tests-report.sh    | 590 ++++++++++++
 .../src/test/bootstrap/aws/tests/tests-run.sh   | 715 +++++++++++++++
 .../CassandraDirectPersistenceLoadTest.java     | 107 +++
 .../tests/CassandraDirectPersistenceTest.java   | 371 ++++++++
 .../apache/ignite/tests/DDLGeneratorTest.java   |  43 +
 .../tests/IgnitePersistentStoreLoadTest.java    | 111 +++
 .../ignite/tests/IgnitePersistentStoreTest.java | 369 ++++++++
 .../org/apache/ignite/tests/load/Generator.java |  27 +
 .../apache/ignite/tests/load/IntGenerator.java  |  33 +
 .../ignite/tests/load/LoadTestDriver.java       | 238 +++++
 .../apache/ignite/tests/load/LongGenerator.java |  28 +
 .../ignite/tests/load/PersonGenerator.java      |  43 +
 .../ignite/tests/load/PersonIdGenerator.java    |  31 +
 .../ignite/tests/load/StringGenerator.java      |  28 +
 .../org/apache/ignite/tests/load/Worker.java    | 429 +++++++++
 .../tests/load/cassandra/BulkReadWorker.java    |  63 ++
 .../tests/load/cassandra/BulkWriteWorker.java   |  52 ++
 .../ignite/tests/load/cassandra/ReadWorker.java |  51 ++
 .../tests/load/cassandra/WriteWorker.java       |  51 ++
 .../tests/load/ignite/BulkReadWorker.java       |  52 ++
 .../tests/load/ignite/BulkWriteWorker.java      |  52 ++
 .../ignite/tests/load/ignite/ReadWorker.java    |  51 ++
 .../ignite/tests/load/ignite/WriteWorker.java   |  51 ++
 .../org/apache/ignite/tests/pojos/Person.java   | 246 +++++
 .../org/apache/ignite/tests/pojos/PersonId.java | 110 +++
 .../ignite/tests/utils/CacheStoreHelper.java    |  64 ++
 .../tests/utils/CassandraAdminCredentials.java  |  36 +
 .../ignite/tests/utils/CassandraHelper.java     | 358 ++++++++
 .../utils/CassandraRegularCredentials.java      |  36 +
 .../ignite/tests/utils/TestCacheSession.java    |  91 ++
 .../apache/ignite/tests/utils/TestsHelper.java  | 375 ++++++++
 .../src/test/resources/log4j.properties         | 119 +++
 .../tests/cassandra/connection-settings.xml     |  48 +
 .../tests/cassandra/connection.properties       |  17 +
 .../tests/cassandra/credentials.properties      |  22 +
 .../tests/cassandra/embedded-cassandra.yaml     | 119 +++
 .../ignite/tests/cassandra/keyspaces.properties |  17 +
 .../tests/persistence/blob/ignite-config.xml    | 100 +++
 .../persistence/blob/persistence-settings-1.xml |  21 +
 .../persistence/blob/persistence-settings-2.xml |  21 +
 .../persistence/blob/persistence-settings-3.xml |  29 +
 .../tests/persistence/pojo/ignite-config.xml    | 119 +++
 .../persistence/pojo/persistence-settings-1.xml |  21 +
 .../persistence/pojo/persistence-settings-2.xml |  21 +
 .../persistence/pojo/persistence-settings-3.xml | 173 ++++
 .../persistence/primitive/ignite-config.xml     | 100 +++
 .../primitive/ignite-remote-client-config.xml   |  95 ++
 .../primitive/ignite-remote-server-config.xml   | 106 +++
 .../primitive/persistence-settings-1.xml        |  21 +
 .../primitive/persistence-settings-2.xml        |  21 +
 .../src/test/resources/tests.properties         |  50 ++
 .../src/test/scripts/cassandra-load-tests.bat   |  41 +
 .../src/test/scripts/cassandra-load-tests.sh    |  39 +
 .../src/test/scripts/ignite-load-tests.bat      |  41 +
 .../src/test/scripts/ignite-load-tests.sh       |  39 +
 modules/cassandra/src/test/scripts/jvm-opt.sh   |  21 +
 modules/cassandra/src/test/scripts/jvm-opts.bat |  24 +
 pom.xml                                         |   9 +
 104 files changed, 16747 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/README.txt
----------------------------------------------------------------------
diff --git a/modules/cassandra/README.txt b/modules/cassandra/README.txt
new file mode 100644
index 0000000..cc2134d
--- /dev/null
+++ b/modules/cassandra/README.txt
@@ -0,0 +1,32 @@
+Apache Ignite Cassandra Module
+------------------------
+
+Apache Ignite Cassandra module provides CacheStore implementation backed by Cassandra database.
+
+To enable Cassandra module when starting a standalone node, move 'optional/ignite-cassandra' folder to
+'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will
+be added to classpath in this case.
+
+Importing Cassandra Module In Maven Project
+-------------------------------------
+
+If you are using Maven to manage dependencies of your project, you can add Cassandra module
+dependency like this (replace '${ignite.version}' with actual Ignite version you are
+interested in):
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                        http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    ...
+    <dependencies>
+        ...
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-cassandra</artifactId>
+            <version>${ignite.version}</version>
+        </dependency>
+        ...
+    </dependencies>
+    ...
+</project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/licenses/apache-2.0.txt
----------------------------------------------------------------------
diff --git a/modules/cassandra/licenses/apache-2.0.txt b/modules/cassandra/licenses/apache-2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/cassandra/licenses/apache-2.0.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/pom.xml b/modules/cassandra/pom.xml
new file mode 100644
index 0000000..d2a7f25
--- /dev/null
+++ b/modules/cassandra/pom.xml
@@ -0,0 +1,340 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    POM file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
+    <artifactId>ignite-cassandra</artifactId>
+    <version>1.6.0-SNAPSHOT</version>
+    <url>http://ignite.apache.org</url>
+
+    <properties>
+        <commons-beanutils.version>1.8.3</commons-beanutils.version>
+        <cassandra-driver.version>3.0.0</cassandra-driver.version>
+        <cassandra-all.version>3.3</cassandra-all.version>
+        <kryo.version>3.0.3</kryo.version>
+        <reflectasm.version>1.10.1</reflectasm.version>
+        <minlog.version>1.3.0</minlog.version>
+        <asm.version>5.0.3</asm.version>
+        <objenesis.version>2.1</objenesis.version>
+        <netty-handler.version>4.0.27.Final</netty-handler.version>
+        <netty-buffer.version>4.0.27.Final</netty-buffer.version>
+        <netty-common.version>4.0.27.Final</netty-common.version>
+        <netty-transport.version>4.0.27.Final</netty-transport.version>
+        <netty-codec.version>4.0.27.Final</netty-codec.version>
+        <guava.version>19.0</guava.version>
+        <metrics-core.version>3.0.2</metrics-core.version>
+    </properties>
+
+    <dependencies>
+        <!-- Apache commons -->
+        <dependency>
+            <groupId>commons-beanutils</groupId>
+            <artifactId>commons-beanutils</artifactId>
+            <version>${commons-beanutils.version}</version>
+        </dependency>
+
+        <!-- Kryo and required dependencies -->
+        <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>kryo</artifactId>
+            <version>${kryo.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>reflectasm</artifactId>
+            <version>${reflectasm.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.esotericsoftware</groupId>
+            <artifactId>minlog</artifactId>
+            <version>${minlog.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.ow2.asm</groupId>
+            <artifactId>asm</artifactId>
+            <version>${asm.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.objenesis</groupId>
+            <artifactId>objenesis</artifactId>
+            <version>${objenesis.version}</version>
+        </dependency>
+
+        <!-- Ignite -->
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-log4j</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- Cassandra and required dependencies -->
+        <dependency>
+            <groupId>com.datastax.cassandra</groupId>
+            <artifactId>cassandra-driver-core</artifactId>
+            <version>${cassandra-driver.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-handler</artifactId>
+            <version>${netty-handler.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-buffer</artifactId>
+            <version>${netty-buffer.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-common</artifactId>
+            <version>${netty-common.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-transport</artifactId>
+            <version>${netty-transport.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>io.netty</groupId>
+            <artifactId>netty-codec</artifactId>
+            <version>${netty-codec.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${guava.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.codahale.metrics</groupId>
+            <artifactId>metrics-core</artifactId>
+            <version>${metrics-core.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.cassandra</groupId>
+            <artifactId>cassandra-all</artifactId>
+            <version>${cassandra-all.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                    <groupId>org.slf4j</groupId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!-- Apache log4j -->
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.2</version>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                    <compilerVersion>1.7</compilerVersion>
+                    <encoding>UTF-8</encoding>
+                    <fork>true</fork>
+                    <debug>false</debug>
+                    <debuglevel>lines,vars,source</debuglevel>
+                    <meminitial>256</meminitial>
+                    <maxmem>512</maxmem>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.10</version>
+                <executions>
+                    <execution>
+                        <id>copy-all-dependencies</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${project.build.directory}/tests-package/lib</outputDirectory>
+                            <overWriteReleases>false</overWriteReleases>
+                            <overWriteSnapshots>false</overWriteSnapshots>
+                            <overWriteIfNewer>true</overWriteIfNewer>
+                        </configuration>
+                    </execution>
+<!-- -->
+                    <execution>
+                        <id>copy-main-dependencies</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${project.build.directory}/libs</outputDirectory>
+                            <overWriteReleases>false</overWriteReleases>
+                            <overWriteSnapshots>false</overWriteSnapshots>
+                            <overWriteIfNewer>true</overWriteIfNewer>
+                            <excludeTransitive>true</excludeTransitive>
+                            <excludeGroupIds>
+                                org.apache.ignite,org.springframework,org.gridgain
+                            </excludeGroupIds>
+                            <excludeArtifactIds>
+                                commons-logging,slf4j-api,cache-api,slf4j-api,aopalliance
+                            </excludeArtifactIds>
+                            <includeScope>runtime</includeScope>
+                        </configuration>
+                    </execution>
+<!-- -->
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <version>1.8</version>
+                <dependencies>
+                    <dependency>
+                        <groupId>ant-contrib</groupId>
+                        <artifactId>ant-contrib</artifactId>
+                        <version>1.0b3</version>
+                        <exclusions>
+                            <exclusion>
+                                <groupId>ant</groupId>
+                                <artifactId>ant</artifactId>
+                            </exclusion>
+                        </exclusions>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <id>package-tests</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                        <configuration>
+                            <target>
+                                <taskdef resource="net/sf/antcontrib/antlib.xml"/>
+                                <if>
+                                    <available file="${project.build.directory}/test-classes" type="dir"/>
+                                    <then>
+                                        <copy todir="${project.build.directory}/tests-package/lib">
+                                            <fileset dir="${project.build.directory}">
+                                                <include name="*.jar"/>
+                                            </fileset>
+                                        </copy>
+
+                                        <jar destfile="${project.build.directory}/tests-package/lib/${project.artifactId}-${project.version}-tests.jar">
+                                            <fileset dir="${project.build.directory}/test-classes">
+                                                <include name="**/*.class"/>
+                                            </fileset>
+                                        </jar>
+
+                                        <copy todir="${project.build.directory}/tests-package/settings">
+                                            <fileset dir="${project.build.directory}/test-classes">
+                                                <include name="**/*.properties"/>
+                                                <include name="**/*.xml"/>
+                                            </fileset>
+                                        </copy>
+
+                                        <copy todir="${project.build.directory}/tests-package">
+                                            <fileset dir="${project.build.testSourceDirectory}/../scripts">
+                                                <include name="**/*"/>
+                                            </fileset>
+                                        </copy>
+
+                                        <fixcrlf srcdir="${project.build.directory}/tests-package" eol="lf" eof="remove">
+                                            <include name="*.sh"/>
+                                        </fixcrlf>
+
+                                        <copy todir="${project.build.directory}/tests-package" >
+                                            <fileset dir="${project.build.testSourceDirectory}/..">
+                                                <include name="bootstrap/**"/>
+                                            </fileset>
+                                        </copy>
+
+                                        <fixcrlf srcdir="${project.build.directory}/tests-package/bootstrap" eol="lf" eof="remove">
+                                            <include name="**"/>
+                                        </fixcrlf>
+
+                                        <zip destfile="${project.build.directory}/ignite-cassandra-tests-${project.version}.zip"
+                                             compress="true" whenempty="create" level="9" encoding="UTF-8"
+                                             useLanguageEncodingFlag="true" createUnicodeExtraFields="not-encodeable">
+
+                                            <zipfileset dir="${project.build.directory}/tests-package" prefix="ignite-cassandra-tests">
+                                                <exclude name="**/*.sh"/>
+                                            </zipfileset>
+
+                                            <zipfileset dir="${project.build.directory}/tests-package" prefix="ignite-cassandra-tests" filemode="555">
+                                                <include name="**/*.sh"/>
+                                            </zipfileset>
+                                        </zip>
+                                    </then>
+                                </if>
+                            </target>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+        </plugins>
+    </build>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
new file mode 100644
index 0000000..f7e7917
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
@@ -0,0 +1,409 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import javax.cache.Cache;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreSession;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
+import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
+import org.apache.ignite.cache.store.cassandra.session.ExecutionAssistant;
+import org.apache.ignite.cache.store.cassandra.session.GenericBatchExecutionAssistant;
+import org.apache.ignite.cache.store.cassandra.session.LoadCacheCustomQueryWorker;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.logger.NullLogger;
+import org.apache.ignite.resources.CacheStoreSessionResource;
+import org.apache.ignite.resources.LoggerResource;
+
+/**
+ * Implementation of {@link CacheStore} backed by Cassandra database.
+ *
+ * @param <K> Ignite cache key type.
+ * @param <V> Ignite cache value type.
+ */
+public class CassandraCacheStore<K, V> implements CacheStore<K, V> {
+    /** Connection attribute property name. */
+    private static final String ATTR_CONN_PROP = "CASSANDRA_STORE_CONNECTION";
+
+    /** Auto-injected store session. */
+    @CacheStoreSessionResource
+    private CacheStoreSession storeSes;
+
+    /** Auto-injected logger instance. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Cassandra data source. */
+    private DataSource dataSrc;
+
+    /** Max workers thread count. These threads are responsible for load cache. */
+    private int maxPoolSize = Runtime.getRuntime().availableProcessors();
+
+    /** Controller component responsible for serialization logic. */
+    private PersistenceController controller;
+
+    /**
+     * Store constructor.
+     *
+     * @param dataSrc Data source.
+     * @param settings Persistence settings for Ignite key and value objects.
+     * @param maxPoolSize Max workers thread count.
+     */
+    public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings settings, int maxPoolSize) {
+        this.dataSrc = dataSrc;
+        this.controller = new PersistenceController(settings);
+        this.maxPoolSize = maxPoolSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadCache(IgniteBiInClosure<K, V> clo, Object... args) throws CacheLoaderException {
+        if (clo == null || args == null || args.length == 0)
+            return;
+
+        ExecutorService pool = null;
+
+        Collection<Future<?>> futs = new ArrayList<>(args.length);
+
+        try {
+            pool = Executors.newFixedThreadPool(maxPoolSize);
+
+            CassandraSession ses = getCassandraSession();
+
+            for (Object obj : args) {
+                if (obj == null || !(obj instanceof String) || !((String)obj).trim().toLowerCase().startsWith("select"))
+                    continue;
+
+                futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo)));
+            }
+
+            for (Future<?> fut : futs)
+                U.get(fut);
+
+            if (log != null && log.isDebugEnabled() && storeSes != null)
+                log.debug("Cache loaded from db: " + storeSes.cacheName());
+        }
+        catch (IgniteCheckedException e) {
+            if (storeSes != null)
+                throw new CacheLoaderException("Failed to load Ignite cache: " + storeSes.cacheName(), e.getCause());
+            else
+                throw new CacheLoaderException("Failed to load cache", e.getCause());
+        }
+        finally {
+            U.shutdownNow(getClass(), pool, log);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sessionEnd(boolean commit) throws CacheWriterException {
+        if (storeSes == null || storeSes.transaction() == null)
+            return;
+
+        CassandraSession cassandraSes = (CassandraSession) storeSes.properties().remove(ATTR_CONN_PROP);
+
+        U.closeQuiet(cassandraSes);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"unchecked"})
+    @Override public V load(final K key) throws CacheLoaderException {
+        if (key == null)
+            return null;
+
+        CassandraSession ses = getCassandraSession();
+
+        try {
+            return ses.execute(new ExecutionAssistant<V>() {
+                @Override public boolean tableExistenceRequired() {
+                    return false;
+                }
+
+                @Override public String getStatement() {
+                    return controller.getLoadStatement(false);
+                }
+
+                @Override public BoundStatement bindStatement(PreparedStatement statement) {
+                    return controller.bindKey(statement, key);
+                }
+
+                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+                    return controller.getPersistenceSettings();
+                }
+
+                @Override public String operationName() {
+                    return "READ";
+                }
+
+                @Override public V process(Row row) {
+                    return row == null ? null : (V)controller.buildValueObject(row);
+                }
+            });
+        }
+        finally {
+            closeCassandraSession(ses);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Map<K, V> loadAll(Iterable<? extends K> keys) throws CacheLoaderException {
+        if (keys == null || !keys.iterator().hasNext())
+            return new HashMap<>();
+
+        CassandraSession ses = getCassandraSession();
+
+        try {
+            return ses.execute(new GenericBatchExecutionAssistant<Map<K, V>, K>() {
+                private Map<K, V> data = new HashMap<>();
+
+                /** {@inheritDoc} */
+                @Override public String getStatement() {
+                    return controller.getLoadStatement(true);
+                }
+
+                /** {@inheritDoc} */
+                @Override  public BoundStatement bindStatement(PreparedStatement statement, K key) {
+                    return controller.bindKey(statement, key);
+                }
+
+                /** {@inheritDoc} */
+                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+                    return controller.getPersistenceSettings();
+                }
+
+                /** {@inheritDoc} */
+                @Override public String operationName() {
+                    return "BULK_READ";
+                }
+
+                /** {@inheritDoc} */
+                @Override public Map<K, V> processedData() {
+                    return data;
+                }
+
+                /** {@inheritDoc} */
+                @Override protected void process(Row row) {
+                    data.put((K)controller.buildKeyObject(row), (V)controller.buildValueObject(row));
+                }
+            }, keys);
+        }
+        finally {
+            closeCassandraSession(ses);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(final Cache.Entry<? extends K, ? extends V> entry) throws CacheWriterException {
+        if (entry == null || entry.getKey() == null)
+            return;
+
+        CassandraSession ses = getCassandraSession();
+
+        try {
+            ses.execute(new ExecutionAssistant<Void>() {
+                @Override public boolean tableExistenceRequired() {
+                    return true;
+                }
+
+                @Override public String getStatement() {
+                    return controller.getWriteStatement();
+                }
+
+                @Override public BoundStatement bindStatement(PreparedStatement statement) {
+                    return controller.bindKeyValue(statement, entry.getKey(), entry.getValue());
+                }
+
+                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+                    return controller.getPersistenceSettings();
+                }
+
+                @Override public String operationName() {
+                    return "WRITE";
+                }
+
+                @Override public Void process(Row row) {
+                    return null;
+                }
+            });
+        }
+        finally {
+            closeCassandraSession(ses);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeAll(Collection<Cache.Entry<? extends K, ? extends V>> entries) throws CacheWriterException {
+        if (entries == null || entries.isEmpty())
+            return;
+
+        CassandraSession ses = getCassandraSession();
+
+        try {
+            ses.execute(new GenericBatchExecutionAssistant<Void, Cache.Entry<? extends K, ? extends V>>() {
+                /** {@inheritDoc} */
+                @Override public String getStatement() {
+                    return controller.getWriteStatement();
+                }
+
+                /** {@inheritDoc} */
+                @Override public BoundStatement bindStatement(PreparedStatement statement,
+                    Cache.Entry<? extends K, ? extends V> entry) {
+                    return controller.bindKeyValue(statement, entry.getKey(), entry.getValue());
+                }
+
+                /** {@inheritDoc} */
+                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+                    return controller.getPersistenceSettings();
+                }
+
+                /** {@inheritDoc} */
+                @Override public String operationName() {
+                    return "BULK_WRITE";
+                }
+
+                /** {@inheritDoc} */
+                @Override public boolean tableExistenceRequired() {
+                    return true;
+                }
+            }, entries);
+        }
+        finally {
+            closeCassandraSession(ses);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void delete(final Object key) throws CacheWriterException {
+        if (key == null)
+            return;
+
+        CassandraSession ses = getCassandraSession();
+
+        try {
+            ses.execute(new ExecutionAssistant<Void>() {
+                @Override public boolean tableExistenceRequired() {
+                    return false;
+                }
+
+                @Override public String getStatement() {
+                    return controller.getDeleteStatement();
+                }
+
+                @Override public BoundStatement bindStatement(PreparedStatement statement) {
+                    return controller.bindKey(statement, key);
+                }
+
+
+                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+                    return controller.getPersistenceSettings();
+                }
+
+                @Override public String operationName() {
+                    return "DELETE";
+                }
+
+                @Override public Void process(Row row) {
+                    return null;
+                }
+            });
+        }
+        finally {
+            closeCassandraSession(ses);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void deleteAll(Collection<?> keys) throws CacheWriterException {
+        if (keys == null || keys.isEmpty())
+            return;
+
+        CassandraSession ses = getCassandraSession();
+
+        try {
+            ses.execute(new GenericBatchExecutionAssistant<Void, Object>() {
+                /** {@inheritDoc} */
+                @Override public String getStatement() {
+                    return controller.getDeleteStatement();
+                }
+
+                /** {@inheritDoc} */
+                @Override public BoundStatement bindStatement(PreparedStatement statement, Object key) {
+                    return controller.bindKey(statement, key);
+                }
+
+                /** {@inheritDoc} */
+                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+                    return controller.getPersistenceSettings();
+                }
+
+                @Override public String operationName() {
+                    return "BULK_DELETE";
+                }
+            }, keys);
+        }
+        finally {
+            closeCassandraSession(ses);
+        }
+    }
+
+    /**
+     * Gets Cassandra session wrapper or creates new if it doesn't exist.
+     * This wrapper hides all the low-level Cassandra interaction details by providing only high-level methods.
+     *
+     * @return Cassandra session wrapper.
+     */
+    private CassandraSession getCassandraSession() {
+        if (storeSes == null || storeSes.transaction() == null)
+            return dataSrc.session(log != null ? log : new NullLogger());
+
+        CassandraSession ses = (CassandraSession) storeSes.properties().get(ATTR_CONN_PROP);
+
+        if (ses == null) {
+            ses = dataSrc.session(log != null ? log : new NullLogger());
+            storeSes.properties().put(ATTR_CONN_PROP, ses);
+        }
+
+        return ses;
+    }
+
+    /**
+     * Releases Cassandra related resources.
+     *
+     * @param ses Cassandra session wrapper.
+     */
+    private void closeCassandraSession(CassandraSession ses) {
+        if (ses != null && (storeSes == null || storeSes.transaction() == null))
+            U.closeQuiet(ses);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java
new file mode 100644
index 0000000..7584dfb
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra;
+
+import javax.cache.configuration.Factory;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.internal.IgniteComponentType;
+import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
+import org.apache.ignite.resources.SpringApplicationContextResource;
+
+/**
+ * Factory class to instantiate {@link CassandraCacheStore}.
+ *
+ * @param <K> Ignite cache key type
+ * @param <V> Ignite cache value type
+ */
+public class CassandraCacheStoreFactory<K, V> implements Factory<CassandraCacheStore<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Auto-injected Spring ApplicationContext resource. */
+    @SpringApplicationContextResource
+    private Object appCtx;
+
+    /** Name of data source bean. */
+    private String dataSrcBean;
+
+    /** Name of persistence settings bean. */
+    private String persistenceSettingsBean;
+
+    /** Data source. */
+    private transient DataSource dataSrc;
+
+    /** Persistence settings. */
+    private KeyValuePersistenceSettings persistenceSettings;
+
+    /** Max workers thread count. These threads are responsible for load cache. */
+    private int maxPoolSize = Runtime.getRuntime().availableProcessors();
+
+    /** {@inheritDoc} */
+    @Override public CassandraCacheStore<K, V> create() {
+        return new CassandraCacheStore<>(getDataSource(), getPersistenceSettings(), getMaxPoolSize());
+    }
+
+    /**
+     * Sets data source.
+     *
+     * @param dataSrc Data source.
+     *
+     * @return {@code This} for chaining.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public CassandraCacheStoreFactory<K, V> setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+
+        return this;
+    }
+
+    /**
+     * Sets data source bean name.
+     *
+     * @param beanName Data source bean name.
+     * @return {@code This} for chaining.
+     */
+    public CassandraCacheStoreFactory<K, V> setDataSourceBean(String beanName) {
+        this.dataSrcBean = beanName;
+
+        return this;
+    }
+
+    /**
+     * Sets persistence settings.
+     *
+     * @param settings Persistence settings.
+     * @return {@code This} for chaining.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public CassandraCacheStoreFactory<K, V> setPersistenceSettings(KeyValuePersistenceSettings settings) {
+        this.persistenceSettings = settings;
+
+        return this;
+    }
+
+    /**
+     * Sets persistence settings bean name.
+     *
+     * @param beanName Persistence settings bean name.
+     * @return {@code This} for chaining.
+     */
+    public CassandraCacheStoreFactory<K, V> setPersistenceSettingsBean(String beanName) {
+        this.persistenceSettingsBean = beanName;
+
+        return this;
+    }
+
+    /**
+     * @return Data source.
+     */
+    private DataSource getDataSource() {
+        if (dataSrc != null)
+            return dataSrc;
+
+        if (dataSrcBean == null)
+            throw new IllegalStateException("Either DataSource bean or DataSource itself should be specified");
+
+        if (appCtx == null) {
+            throw new IllegalStateException("Failed to get Cassandra DataSource cause Spring application " +
+                "context wasn't injected into CassandraCacheStoreFactory");
+        }
+
+        Object obj = loadSpringContextBean(appCtx, dataSrcBean);
+
+        if (!(obj instanceof DataSource))
+            throw new IllegalStateException("Incorrect connection bean '" + dataSrcBean + "' specified");
+
+        return dataSrc = (DataSource)obj;
+    }
+
+    /**
+     * @return Persistence settings.
+     */
+    private KeyValuePersistenceSettings getPersistenceSettings() {
+        if (persistenceSettings != null)
+            return persistenceSettings;
+
+        if (persistenceSettingsBean == null) {
+            throw new IllegalStateException("Either persistence settings bean or persistence settings itself " +
+                "should be specified");
+        }
+
+        if (appCtx == null) {
+            throw new IllegalStateException("Failed to get Cassandra persistence settings cause Spring application " +
+                "context wasn't injected into CassandraCacheStoreFactory");
+        }
+
+        Object obj = loadSpringContextBean(appCtx, persistenceSettingsBean);
+
+        if (!(obj instanceof KeyValuePersistenceSettings)) {
+            throw new IllegalStateException("Incorrect persistence settings bean '" +
+                persistenceSettingsBean + "' specified");
+        }
+
+        return persistenceSettings = (KeyValuePersistenceSettings)obj;
+    }
+
+    /**
+     * Get maximum workers thread count. These threads are responsible for queries execution.
+     *
+     * @return Maximum workers thread count.
+     */
+    public int getMaxPoolSize() {
+        return maxPoolSize;
+    }
+
+    /**
+     * Set Maximum workers thread count. These threads are responsible for queries execution.
+     *
+     * @param maxPoolSize Max workers thread count.
+     * @return {@code This} for chaining.
+     */
+    public CassandraCacheStoreFactory<K, V> setMaxPoolSize(int maxPoolSize) {
+        this.maxPoolSize = maxPoolSize;
+
+        return this;
+    }
+
+    /**
+     * Loads bean from Spring ApplicationContext.
+     *
+     * @param appCtx Application context.
+     * @param beanName Bean name to load.
+     * @return Loaded bean.
+     */
+    private Object loadSpringContextBean(Object appCtx, String beanName) {
+        try {
+            IgniteSpringHelper spring = IgniteComponentType.SPRING.create(false);
+            return spring.loadBeanFromAppContext(appCtx, beanName);
+        }
+        catch (Exception e) {
+            throw new IgniteException("Failed to load bean in application context [beanName=" + beanName + ", igniteConfig=" + appCtx + ']', e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java
new file mode 100644
index 0000000..11f8145
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/bean/CassandraLifeCycleBean.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.bean;
+
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.lifecycle.LifecycleBean;
+import org.apache.ignite.lifecycle.LifecycleEventType;
+import org.apache.ignite.resources.LoggerResource;
+
+/**
+ * Implementation of {@link LifecycleBean} to start embedded Cassandra instance on Ignite cluster startup
+ */
+public class CassandraLifeCycleBean implements LifecycleBean {
+    /** System property specifying Cassandra jmx port */
+    private static final String CASSANDRA_JMX_PORT_PROP = "cassandra.jmx.local.port";
+
+    /** System property specifying Cassandra YAML config file */
+    private static final String CASSANDRA_CONFIG_PROP = "cassandra.config";
+
+    /** Prefix for file path syntax */
+    private static final String FILE_PREFIX = "file:///";
+
+    /** Auto-injected logger instance. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Instance of embedded Cassandra database */
+    private CassandraDaemon embeddedCassandraDaemon;
+
+    /** JMX port for embedded Cassandra instance */
+    private String jmxPort;
+
+    /** YAML config file for embedded Cassandra */
+    private String cassandraCfgFile;
+
+    /**
+     * Returns JMX port for embedded Cassandra
+     * @return JMX port
+     */
+    public String getJmxPort() {
+        return jmxPort;
+    }
+
+    /**
+     * Setter for embedded Cassandra JMX port
+     * @param jmxPort embedded Cassandra JMX port
+     */
+    public void setJmxPort(String jmxPort) {
+        this.jmxPort = jmxPort;
+    }
+
+    /**
+     * Returns embedded Cassandra YAML config file
+     * @return YAML config file
+     */
+    public String getCassandraConfigFile() {
+        return cassandraCfgFile;
+    }
+
+    /**
+     * Setter for embedded Cassandra YAML config file
+     * @param cassandraCfgFile YAML config file
+     */
+    public void setCassandraConfigFile(String cassandraCfgFile) {
+        this.cassandraCfgFile = cassandraCfgFile;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLifecycleEvent(LifecycleEventType evt) {
+        if (evt == LifecycleEventType.BEFORE_NODE_START)
+            startEmbeddedCassandra();
+        else if (evt == LifecycleEventType.BEFORE_NODE_STOP)
+            stopEmbeddedCassandra();
+    }
+
+    /**
+     * Starts embedded Cassandra instance
+     */
+    private void startEmbeddedCassandra() {
+        if (log != null) {
+            log.info("-------------------------------");
+            log.info("| Starting embedded Cassandra |");
+            log.info("-------------------------------");
+        }
+
+        try {
+            if (jmxPort != null)
+                System.setProperty(CASSANDRA_JMX_PORT_PROP, jmxPort);
+
+            if (cassandraCfgFile != null)
+                System.setProperty(CASSANDRA_CONFIG_PROP, FILE_PREFIX + cassandraCfgFile);
+
+            embeddedCassandraDaemon = new CassandraDaemon(true);
+            embeddedCassandraDaemon.init(null);
+            embeddedCassandraDaemon.start();
+        }
+        catch (Exception e) {
+            throw new RuntimeException("Failed to start embedded Cassandra", e);
+        }
+
+        if (log != null) {
+            log.info("------------------------------");
+            log.info("| Embedded Cassandra started |");
+            log.info("------------------------------");
+        }
+    }
+
+    /**
+     * Stops embedded Cassandra instance
+     */
+    private void stopEmbeddedCassandra() {
+        if (log != null) {
+            log.info("-------------------------------");
+            log.info("| Stopping embedded Cassandra |");
+            log.info("-------------------------------");
+        }
+
+        if (embeddedCassandraDaemon != null) {
+            try {
+                embeddedCassandraDaemon.deactivate();
+            }
+            catch (Throwable e) {
+                throw new RuntimeException("Failed to stop embedded Cassandra", e);
+            }
+        }
+
+        if (log != null) {
+            log.info("------------------------------");
+            log.info("| Embedded Cassandra stopped |");
+            log.info("------------------------------");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/CassandraHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/CassandraHelper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/CassandraHelper.java
new file mode 100644
index 0000000..d3bff7f
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/CassandraHelper.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.common;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.datastax.driver.core.exceptions.ReadTimeoutException;
+import java.util.regex.Pattern;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Helper class providing methods to work with Cassandra session and exceptions
+ */
+public class CassandraHelper {
+    /** Cassandra error message if specified keyspace doesn't exist. */
+    private static final Pattern KEYSPACE_EXIST_ERROR1 = Pattern.compile("Keyspace [0-9a-zA-Z_]+ does not exist");
+
+    /** Cassandra error message if trying to create table inside nonexistent keyspace. */
+    private static final Pattern KEYSPACE_EXIST_ERROR2 = Pattern.compile("Cannot add table '[0-9a-zA-Z_]+' to non existing keyspace.*");
+
+    /** Cassandra error message if specified table doesn't exist. */
+    private static final Pattern TABLE_EXIST_ERROR = Pattern.compile("unconfigured table [0-9a-zA-Z_]+");
+
+    /** Cassandra error message if trying to use prepared statement created from another session. */
+    private static final String PREP_STATEMENT_CLUSTER_INSTANCE_ERROR = "You may have used a PreparedStatement that " +
+        "was created with another Cluster instance";
+
+    /** Closes Cassandra driver session. */
+    public static void closeSession(Session driverSes) {
+        if (driverSes == null)
+            return;
+
+        Cluster cluster = driverSes.getCluster();
+
+        if (!driverSes.isClosed())
+            U.closeQuiet(driverSes);
+
+        if (!cluster.isClosed())
+            U.closeQuiet(cluster);
+    }
+
+    /**
+     * Checks if Cassandra keyspace absence error occur.
+     *
+     * @param e Exception to check.
+     * @return {@code true} in case of keyspace absence error.
+     */
+    public static boolean isKeyspaceAbsenceError(Throwable e) {
+        while (e != null) {
+            if (e instanceof InvalidQueryException &&
+                (KEYSPACE_EXIST_ERROR1.matcher(e.getMessage()).matches() ||
+                    KEYSPACE_EXIST_ERROR2.matcher(e.getMessage()).matches()))
+                return true;
+
+            e = e.getCause();
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks if Cassandra table absence error occur.
+     *
+     * @param e Exception to check.
+     * @return {@code true} in case of table absence error.
+     */
+    public static boolean isTableAbsenceError(Throwable e) {
+        while (e != null) {
+            if (e instanceof InvalidQueryException &&
+                (TABLE_EXIST_ERROR.matcher(e.getMessage()).matches() ||
+                    KEYSPACE_EXIST_ERROR1.matcher(e.getMessage()).matches() ||
+                    KEYSPACE_EXIST_ERROR2.matcher(e.getMessage()).matches()))
+                return true;
+
+            e = e.getCause();
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks if Cassandra host availability error occur, thus host became unavailable.
+     *
+     * @param e Exception to check.
+     * @return {@code true} in case of host not available error.
+     */
+    public static boolean isHostsAvailabilityError(Throwable e) {
+        while (e != null) {
+            if (e instanceof NoHostAvailableException ||
+                e instanceof ReadTimeoutException)
+                return true;
+
+            e = e.getCause();
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks if Cassandra error occur because of prepared statement created in one session was used in another session.
+     *
+     * @param e Exception to check.
+     * @return {@code true} in case of invalid usage of prepared statement.
+     */
+    public static boolean isPreparedStatementClusterError(Throwable e) {
+        while (e != null) {
+            if (e instanceof InvalidQueryException && e.getMessage().contains(PREP_STATEMENT_CLUSTER_INSTANCE_ERROR))
+                return true;
+
+            e = e.getCause();
+        }
+
+        return false;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/PropertyMappingHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/PropertyMappingHelper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/PropertyMappingHelper.java
new file mode 100644
index 0000000..9053a93
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/PropertyMappingHelper.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.common;
+
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.Row;
+import java.beans.PropertyDescriptor;
+import java.lang.annotation.Annotation;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.commons.beanutils.PropertyUtils;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+
+/**
+ * Helper class providing bunch of methods to discover fields of POJO objects and
+ * map builtin Java types to appropriate Cassandra types.
+ */
+public class PropertyMappingHelper {
+    /** Bytes array Class type. */
+    private static final Class BYTES_ARRAY_CLASS = (new byte[] {}).getClass();
+
+    /** Mapping from Java to Cassandra types. */
+    private static final Map<Class, DataType.Name> JAVA_TO_CASSANDRA_MAPPING = new HashMap<Class, DataType.Name>() {{
+        put(String.class, DataType.Name.TEXT);
+        put(Integer.class, DataType.Name.INT);
+        put(int.class, DataType.Name.INT);
+        put(Short.class, DataType.Name.INT);
+        put(short.class, DataType.Name.INT);
+        put(Long.class, DataType.Name.BIGINT);
+        put(long.class, DataType.Name.BIGINT);
+        put(Double.class, DataType.Name.DOUBLE);
+        put(double.class, DataType.Name.DOUBLE);
+        put(Boolean.class, DataType.Name.BOOLEAN);
+        put(boolean.class, DataType.Name.BOOLEAN);
+        put(Float.class, DataType.Name.FLOAT);
+        put(float.class, DataType.Name.FLOAT);
+        put(ByteBuffer.class, DataType.Name.BLOB);
+        put(BYTES_ARRAY_CLASS, DataType.Name.BLOB);
+        put(BigDecimal.class, DataType.Name.DECIMAL);
+        put(InetAddress.class, DataType.Name.INET);
+        put(Date.class, DataType.Name.TIMESTAMP);
+        put(UUID.class, DataType.Name.UUID);
+        put(BigInteger.class, DataType.Name.VARINT);
+    }};
+
+    /**
+     * Maps Cassandra type to specified Java type.
+     *
+     * @param clazz java class.
+     *
+     * @return Cassandra type.
+     */
+    public static DataType.Name getCassandraType(Class clazz) {
+        return JAVA_TO_CASSANDRA_MAPPING.get(clazz);
+    }
+
+    /**
+     * Returns property descriptor by class property name.
+     *
+     * @param clazz class from which to get property descriptor.
+     * @param prop name of the property.
+     *
+     * @return property descriptor.
+     */
+    public static PropertyDescriptor getPojoPropertyDescriptor(Class clazz, String prop) {
+        List<PropertyDescriptor> descriptors = getPojoPropertyDescriptors(clazz, false);
+
+        if (descriptors == null || descriptors.isEmpty())
+            throw new IllegalArgumentException("POJO class " + clazz.getName() + " doesn't have '" + prop + "' property");
+
+        for (PropertyDescriptor descriptor : descriptors) {
+            if (descriptor.getName().equals(prop))
+                return descriptor;
+        }
+
+        throw new IllegalArgumentException("POJO class " + clazz.getName() + " doesn't have '" + prop + "' property");
+    }
+
+    /**
+     * Extracts all property descriptors from a class.
+     *
+     * @param clazz class which property descriptors should be extracted.
+     * @param primitive boolean flag indicating that only property descriptors for primitive properties should be extracted.
+     *
+     * @return list of class property descriptors
+     */
+    public static List<PropertyDescriptor> getPojoPropertyDescriptors(Class clazz, boolean primitive) {
+        return getPojoPropertyDescriptors(clazz, null, primitive);
+    }
+
+    /**
+     * Extracts all property descriptors having specific annotation from a class.
+     *
+     * @param clazz class which property descriptors should be extracted.
+     * @param annotation annotation to look for.
+     * @param primitive boolean flag indicating that only property descriptors for primitive properties should be extracted.
+     *
+     * @return list of class property descriptors
+     */
+    public static <T extends Annotation> List<PropertyDescriptor> getPojoPropertyDescriptors(Class clazz,
+        Class<T> annotation, boolean primitive) {
+        PropertyDescriptor[] descriptors = PropertyUtils.getPropertyDescriptors(clazz);
+
+        List<PropertyDescriptor> list = new ArrayList<>(descriptors == null ? 1 : descriptors.length);
+
+        if (descriptors == null || descriptors.length == 0)
+            return list;
+
+        for (PropertyDescriptor descriptor : descriptors) {
+            if (descriptor.getReadMethod() == null || descriptor.getWriteMethod() == null ||
+                (primitive && !isPrimitivePropertyDescriptor(descriptor)))
+                continue;
+
+            if (annotation == null || descriptor.getReadMethod().getAnnotation(annotation) != null)
+                list.add(descriptor);
+        }
+
+        return list;
+    }
+
+    /**
+     * Checks if property descriptor describes primitive property (int, boolean, long and etc.)
+     *
+     * @param desc property descriptor.
+     *
+     * @return {@code true} property is primitive
+     */
+    public static boolean isPrimitivePropertyDescriptor(PropertyDescriptor desc) {
+        return PropertyMappingHelper.JAVA_TO_CASSANDRA_MAPPING.containsKey(desc.getPropertyType());
+    }
+
+    /**
+     * Returns value of specific column in the row returned by CQL statement.
+     *
+     * @param row row returned by CQL statement.
+     * @param col column name.
+     * @param clazz java class to which column value should be casted.
+     * @param serializer serializer to use if column stores BLOB otherwise could be null.
+     *
+     * @return row column value.
+     */
+    public static Object getCassandraColumnValue(Row row, String col, Class clazz, Serializer serializer) {
+        if (String.class.equals(clazz))
+            return row.getString(col);
+
+        if (Integer.class.equals(clazz) || int.class.equals(clazz))
+            return row.getInt(col);
+
+        if (Short.class.equals(clazz) || short.class.equals(clazz))
+            return (short)row.getInt(col);
+
+        if (Long.class.equals(clazz) || long.class.equals(clazz))
+            return row.getLong(col);
+
+        if (Double.class.equals(clazz) || double.class.equals(clazz))
+            return row.getDouble(col);
+
+        if (Boolean.class.equals(clazz) || boolean.class.equals(clazz))
+            return row.getBool(col);
+
+        if (Float.class.equals(clazz) || float.class.equals(clazz))
+            return row.getFloat(col);
+
+        if (ByteBuffer.class.equals(clazz))
+            return row.getBytes(col);
+
+        if (PropertyMappingHelper.BYTES_ARRAY_CLASS.equals(clazz)) {
+            ByteBuffer buf = row.getBytes(col);
+
+            return buf == null ? null : buf.array();
+        }
+
+        if (BigDecimal.class.equals(clazz))
+            return row.getDecimal(col);
+
+        if (InetAddress.class.equals(clazz))
+            return row.getInet(col);
+
+        if (Date.class.equals(clazz))
+            return row.getTimestamp(col);
+
+        if (UUID.class.equals(clazz))
+            return row.getUUID(col);
+
+        if (BigInteger.class.equals(clazz))
+            return row.getVarint(col);
+
+        if (serializer == null) {
+            throw new IllegalStateException("Can't deserialize value from '" + col + "' Cassandra column, " +
+                "cause there is no BLOB serializer specified");
+        }
+
+        ByteBuffer buf = row.getBytes(col);
+
+        return buf == null ? null : serializer.deserialize(buf);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/RandomSleeper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/RandomSleeper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/RandomSleeper.java
new file mode 100644
index 0000000..6745a16
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/RandomSleeper.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.common;
+
+import java.util.Random;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+
+/**
+ * Provides sleep method with randomly selected sleep time from specified range and
+ * incrementally shifts sleep time range for each next sleep attempt
+ *
+ */
+public class RandomSleeper {
+    /** */
+    private int min;
+
+    /** */
+    private int max;
+
+    /** */
+    private int incr;
+
+    /** */
+    private IgniteLogger log;
+
+    /** */
+    private Random random = new Random(System.currentTimeMillis());
+
+    /** */
+    private int summary = 0;
+
+    /**
+     * Creates sleeper instance.
+     *
+     * @param min minimum sleep time (in milliseconds)
+     * @param max maximum sleep time (in milliseconds)
+     * @param incr time range shift increment (in milliseconds)
+     */
+    public RandomSleeper(int min, int max, int incr, IgniteLogger log) {
+        if (min <= 0)
+            throw new IllegalArgumentException("Incorrect min time specified: " + min);
+
+        if (max <= min)
+            throw new IllegalArgumentException("Incorrect max time specified: " + max);
+
+        if (incr < 10)
+            throw new IllegalArgumentException("Incorrect increment specified: " + incr);
+
+        this.min = min;
+        this.max = max;
+        this.incr = incr;
+        this.log = log;
+    }
+
+    /**
+     * Sleeps
+     */
+    public void sleep() {
+        try {
+            int timeout = random.nextInt(max - min + 1) + min;
+
+            if (log != null)
+                log.info("Sleeping for " + timeout + "ms");
+
+            Thread.sleep(timeout);
+
+            summary += timeout;
+
+            if (log != null)
+                log.info("Sleep completed");
+        }
+        catch (InterruptedException e) {
+            throw new IgniteException("Random sleep interrupted", e);
+        }
+
+        min += incr;
+        max += incr;
+    }
+
+    /**
+     * Returns summary sleep time.
+     *
+     * @return Summary sleep time in milliseconds.
+     */
+    public int getSleepSummary() {
+        return summary;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/SystemHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/SystemHelper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/SystemHelper.java
new file mode 100644
index 0000000..5d51488
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/common/SystemHelper.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.common;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+/**
+ * Helper class providing system information about the host (ip, hostname, os and etc.)
+ */
+public class SystemHelper {
+    /** System line separator. */
+    public static final String LINE_SEPARATOR = System.getProperty("line.separator");
+
+    /** Host name. */
+    public static final String HOST_NAME;
+
+    /** Host IP address */
+    public static final String HOST_IP;
+
+    static {
+        try {
+            InetAddress addr = InetAddress.getLocalHost();
+            HOST_NAME = addr.getHostName();
+            HOST_IP = addr.getHostAddress();
+        }
+        catch (UnknownHostException e) {
+            throw new IllegalStateException("Failed to get host/ip of current computer", e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/Credentials.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/Credentials.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/Credentials.java
new file mode 100644
index 0000000..e1fd60c
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/Credentials.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.datasource;
+
+/**
+ * Provides credentials for Cassandra (instead of specifying user/password directly in Spring context XML).
+ */
+public interface Credentials {
+    /**
+     * Returns user name
+     *
+     * @return user name
+     */
+    public String getUser();
+
+    /**
+     * Returns password
+     *
+     * @return password
+     */
+    public String getPassword();
+}


[24/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java
new file mode 100644
index 0000000..4e86d74
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import java.beans.PropertyDescriptor;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.w3c.dom.Element;
+
+/**
+ * Descriptor for Ignite key POJO class
+ */
+public class PojoKeyField extends PojoField {
+
+    /**
+     * Specifies sort order for POJO key field
+     */
+    public enum SortOrder {
+        /** Ascending sort order. */
+        ASC,
+        /** Descending sort order. */
+        DESC
+    }
+
+    /** Xml attribute specifying sort order. */
+    private static final String SORT_ATTR = "sort";
+
+    /** Sort order. */
+    private SortOrder sortOrder = null;
+
+    /**
+     * Constructs Ignite cache key POJO object descriptor.
+     *
+     * @param el xml configuration element.
+     * @param pojoCls java class of key POJO field.
+     */
+    public PojoKeyField(Element el, Class pojoCls) {
+        super(el, pojoCls);
+
+        if (el.hasAttribute(SORT_ATTR)) {
+            try {
+                sortOrder = SortOrder.valueOf(el.getAttribute(SORT_ATTR).trim().toUpperCase());
+            }
+            catch (IllegalArgumentException e) {
+                throw new IllegalArgumentException("Incorrect sort order '" + el.getAttribute(SORT_ATTR) + "' specified");
+            }
+        }
+    }
+
+    /**
+     * Constructs Ignite cache key POJO object descriptor.
+     *
+     * @param desc property descriptor.
+     */
+    public PojoKeyField(PropertyDescriptor desc) {
+        super(desc);
+    }
+
+    /**
+     * Returns sort order for the field.
+     *
+     * @return sort order.
+     */
+    public SortOrder getSortOrder() {
+        return sortOrder;
+    }
+
+    /**
+     * Initializes descriptor from {@link QuerySqlField} annotation.
+     *
+     * @param sqlField {@link QuerySqlField} annotation.
+     */
+    protected void init(QuerySqlField sqlField) {
+        if (sqlField.descending())
+            sortOrder = SortOrder.DESC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java
new file mode 100644
index 0000000..c29f1db
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import java.beans.PropertyDescriptor;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.w3c.dom.Element;
+
+/**
+ * Descriptor for Ignite value POJO class
+ */
+public class PojoValueField extends PojoField {
+    /** Xml attribute specifying that Cassandra column is static. */
+    private static final String STATIC_ATTR = "static";
+
+    /** Xml attribute specifying that secondary index should be created for Cassandra column. */
+    private static final String INDEX_ATTR = "index";
+
+    /** Xml attribute specifying secondary index custom class. */
+    private static final String INDEX_CLASS_ATTR = "indexClass";
+
+    /** Xml attribute specifying secondary index options. */
+    private static final String INDEX_OPTIONS_ATTR = "indexOptions";
+
+    /** Indicates if Cassandra column should be indexed. */
+    private Boolean isIndexed;
+
+    /** Custom java class for Cassandra secondary index. */
+    private String idxCls;
+
+    /** Secondary index options. */
+    private String idxOptions;
+
+    /** Indicates if Cassandra column is static. */
+    private Boolean isStatic;
+
+    /**
+     * Constructs Ignite cache value field descriptor.
+     *
+     * @param el field descriptor xml configuration element.
+     * @param pojoCls field java class
+     */
+    public PojoValueField(Element el, Class pojoCls) {
+        super(el, pojoCls);
+
+        if (el.hasAttribute(STATIC_ATTR))
+            isStatic = Boolean.parseBoolean(el.getAttribute(STATIC_ATTR).trim().toLowerCase());
+
+        if (el.hasAttribute(INDEX_ATTR))
+            isIndexed = Boolean.parseBoolean(el.getAttribute(INDEX_ATTR).trim().toLowerCase());
+
+        if (el.hasAttribute(INDEX_CLASS_ATTR))
+            idxCls = el.getAttribute(INDEX_CLASS_ATTR).trim();
+
+        if (el.hasAttribute(INDEX_OPTIONS_ATTR)) {
+            idxOptions = el.getAttribute(INDEX_OPTIONS_ATTR).trim();
+
+            if (!idxOptions.toLowerCase().startsWith("with")) {
+                idxOptions = idxOptions.toLowerCase().startsWith("options") ?
+                    "with " + idxOptions :
+                    "with options = " + idxOptions;
+            }
+        }
+    }
+
+    /**
+     * Constructs Ignite cache value field descriptor.
+     *
+     * @param desc field property descriptor.
+     */
+    public PojoValueField(PropertyDescriptor desc) {
+        super(desc);
+    }
+
+    /**
+     * Returns DDL for Cassandra columns corresponding to POJO field.
+     *
+     * @return columns DDL.
+     */
+    public String getColumnDDL() {
+        String colDDL = super.getColumnDDL();
+
+        if (isStatic != null && isStatic)
+            colDDL = colDDL + " static";
+
+        return colDDL;
+    }
+
+    /**
+     * Indicates if secondary index should be created for the field.
+     *
+     * @return true/false if secondary index should/shouldn't be created for the field.
+     */
+    public boolean isIndexed() {
+        return isIndexed != null && isIndexed;
+    }
+
+    /**
+     * Returns DDL for the field secondary index.
+     *
+     * @param keyspace Cassandra keyspace where index should be created.
+     * @param tbl Cassandra table for which secondary index should be created.
+     *
+     * @return secondary index DDL.
+     */
+    public String getIndexDDL(String keyspace, String tbl) {
+        if (isIndexed == null || !isIndexed)
+            return null;
+
+        StringBuilder builder = new StringBuilder();
+
+        if (idxCls != null)
+            builder.append("create custom index if not exists on ").append(keyspace).append(".").append(tbl);
+        else
+            builder.append("create index if not exists on ").append(keyspace).append(".").append(tbl);
+
+        builder.append(" (").append(getColumn()).append(")");
+
+        if (idxCls != null)
+            builder.append(" using '").append(idxCls).append("'");
+
+        if (idxOptions != null)
+            builder.append(" ").append(idxOptions);
+
+        return builder.append(";").toString();
+    }
+
+    /**
+     * Initializes descriptor from {@link QuerySqlField} annotation.
+     *
+     * @param sqlField {@link QuerySqlField} annotation.
+     */
+    protected void init(QuerySqlField sqlField) {
+        if (sqlField.index())
+            isIndexed = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java
new file mode 100644
index 0000000..877167d
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import java.beans.PropertyDescriptor;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+/**
+ * Stores persistence settings for Ignite cache value
+ */
+public class ValuePersistenceSettings extends PersistenceSettings {
+    /** XML element describing value field settings. */
+    private static final String FIELD_ELEMENT = "field";
+
+    /** Value fields. */
+    private List<PojoField> fields = new LinkedList<>();
+
+    /**
+     * Creates class instance from XML configuration.
+     *
+     * @param el XML element describing value persistence settings.
+     */
+    public ValuePersistenceSettings(Element el) {
+        super(el);
+
+        if (!PersistenceStrategy.POJO.equals(getStrategy()))
+            return;
+
+        NodeList nodes = el.getElementsByTagName(FIELD_ELEMENT);
+
+        fields = detectFields(nodes);
+
+        if (fields.isEmpty())
+            throw new IllegalStateException("Failed to initialize value fields for class '" + getJavaClass().getName() + "'");
+
+        checkDuplicates(fields);
+    }
+
+    /**
+     * @return List of value fields.
+     */
+    public List<PojoField> getFields() {
+        return fields == null ? null : Collections.unmodifiableList(fields);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String defaultColumnName() {
+        return "value";
+    }
+
+    /**
+     * Extracts POJO fields from a list of corresponding XML field nodes.
+     *
+     * @param fieldNodes Field nodes to process.
+     * @return POJO fields list.
+     */
+    private List<PojoField> detectFields(NodeList fieldNodes) {
+        List<PojoField> list = new LinkedList<>();
+
+        if (fieldNodes == null || fieldNodes.getLength() == 0) {
+            List<PropertyDescriptor> primitivePropDescriptors = PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), true);
+            for (PropertyDescriptor descriptor : primitivePropDescriptors)
+                list.add(new PojoValueField(descriptor));
+
+            return list;
+        }
+
+        List<PropertyDescriptor> allPropDescriptors = PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), false);
+
+        int cnt = fieldNodes.getLength();
+
+        for (int i = 0; i < cnt; i++) {
+            PojoValueField field = new PojoValueField((Element)fieldNodes.item(i), getJavaClass());
+
+            PropertyDescriptor desc = findPropertyDescriptor(allPropDescriptors, field.getName());
+
+            if (desc == null) {
+                throw new IllegalArgumentException("Specified POJO field '" + field.getName() +
+                    "' doesn't exist in '" + getJavaClass().getName() + "' class");
+            }
+
+            list.add(field);
+        }
+
+        return list;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java
new file mode 100644
index 0000000..e9f93a0
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.serializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Serializer based on standard Java serialization.
+ */
+public class JavaSerializer implements Serializer {
+    /** */
+    private static final int DFLT_BUFFER_SIZE = 4096;
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer serialize(Object obj) {
+        if (obj == null)
+            return null;
+
+        ByteArrayOutputStream stream = null;
+        ObjectOutputStream out = null;
+
+        try {
+            stream = new ByteArrayOutputStream(DFLT_BUFFER_SIZE);
+
+            out = new ObjectOutputStream(stream);
+            out.writeObject(obj);
+            out.flush();
+
+            return ByteBuffer.wrap(stream.toByteArray());
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to serialize object of the class '" + obj.getClass().getName() + "'", e);
+        }
+        finally {
+            U.closeQuiet(out);
+            U.closeQuiet(stream);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object deserialize(ByteBuffer buf) {
+        ByteArrayInputStream stream = null;
+        ObjectInputStream in = null;
+
+        try {
+            stream = new ByteArrayInputStream(buf.array());
+            in = new ObjectInputStream(stream);
+
+            return in.readObject();
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to deserialize object from byte stream", e);
+        }
+        finally {
+            U.closeQuiet(in);
+            U.closeQuiet(stream);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/KryoSerializer.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/KryoSerializer.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/KryoSerializer.java
new file mode 100644
index 0000000..88379de
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/KryoSerializer.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.serializer;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Serializer based on Kryo serialization.
+ */
+public class KryoSerializer implements Serializer {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final int DFLT_BUFFER_SIZE = 4096;
+
+    /** Thread local instance of {@link com.esotericsoftware.kryo.Kryo} */
+    private transient ThreadLocal<Kryo> kryos = new ThreadLocal<Kryo>() {
+        protected Kryo initialValue() {
+            return new Kryo();
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override public ByteBuffer serialize(Object obj) {
+        if (obj == null)
+            return null;
+
+        ByteArrayOutputStream stream = null;
+
+        Output out = null;
+
+        try {
+            stream = new ByteArrayOutputStream(DFLT_BUFFER_SIZE);
+
+            out = new Output(stream);
+
+            kryos.get().writeClassAndObject(out, obj);
+            out.flush();
+
+            return ByteBuffer.wrap(stream.toByteArray());
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to serialize object of the class '" + obj.getClass().getName() + "'", e);
+        }
+        finally {
+            U.closeQuiet(out);
+            U.closeQuiet(stream);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object deserialize(ByteBuffer buf) {
+        ByteArrayInputStream stream = null;
+        Input in = null;
+
+        try {
+            stream = new ByteArrayInputStream(buf.array());
+            in = new Input(stream);
+
+            return kryos.get().readClassAndObject(in);
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to deserialize object from byte stream", e);
+        }
+        finally {
+            U.closeQuiet(in);
+            U.closeQuiet(stream);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java
new file mode 100644
index 0000000..5b8d542
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.serializer;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+/**
+ * Interface which should be implemented by all serializers responsible
+ * for writing/loading data to/from Cassandra in binary (BLOB) format.
+ */
+public interface Serializer extends Serializable {
+    /**
+     * Serializes object into byte buffer.
+     *
+     * @param obj Object to serialize.
+     * @return Byte buffer with binary data.
+     */
+    public ByteBuffer serialize(Object obj);
+
+    /**
+     * Deserializes object from byte buffer.
+     *
+     * @param buf Byte buffer.
+     * @return Deserialized object.
+     */
+    public Object deserialize(ByteBuffer buf);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java
new file mode 100644
index 0000000..e43db1d
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Provides information for batch operations (loadAll, deleteAll, writeAll) of Ignite cache
+ * backed by {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}.
+ *
+ * @param <R> type of the result returned from batch operation.
+ * @param <V> type of the value used in batch operation.
+ */
+public interface BatchExecutionAssistant<R, V> {
+    /**
+     * Indicates if Cassandra tables existence is required for this batch operation.
+     *
+     * @return {@code true} true if table existence required.
+     */
+    public boolean tableExistenceRequired();
+
+    /**
+     * Returns unbind CLQ statement for to be executed inside batch operation.
+     *
+     * @return Unbind CQL statement.
+     */
+    public String getStatement();
+
+    /**
+     * Binds prepared statement to current Cassandra session.
+     *
+     * @param statement Statement.
+     * @param obj Parameters for statement binding.
+     * @return Bounded statement.
+     */
+    public BoundStatement bindStatement(PreparedStatement statement, V obj);
+
+    /**
+     *  Returns Ignite cache key/value persistence settings.
+     *
+     * @return persistence settings.
+     */
+    public KeyValuePersistenceSettings getPersistenceSettings();
+
+    /**
+     * Display name for the batch operation.
+     *
+     * @return Operation display name.
+     */
+    public String operationName();
+
+    /**
+     * Processes particular row inside batch operation.
+     *
+     * @param row Row to process.
+     * @param seqNum Sequential number of the row.
+     */
+    public void process(Row row, int seqNum);
+
+    /**
+     * Checks if row/object with specified sequential number is already processed.
+     *
+     * @param seqNum object sequential number
+     * @return {@code true} if object is already processed
+     */
+    public boolean alreadyProcessed(int seqNum);
+
+    /**
+     * @return number of processed objects/rows.
+     */
+    public int processedCount();
+
+    /**
+     * @return batch operation result.
+     */
+    public R processedData();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java
new file mode 100644
index 0000000..387c98f
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Statement;
+
+/**
+ * Provides information for loadCache operation of {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}.
+ */
+public interface BatchLoaderAssistant {
+    /**
+     * Returns name of the batch load operation.
+     *
+     * @return operation name.
+     */
+    public String operationName();
+
+    /**
+     * Returns CQL statement to use in batch load operation.
+     *
+     * @return CQL statement for batch load operation.
+     */
+    public Statement getStatement();
+
+    /**
+     * Processes each row returned by batch load operation.
+     *
+     * @param row row selected from Cassandra table.
+     */
+    public void process(Row row);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.java
new file mode 100644
index 0000000..506982f
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import java.io.Closeable;
+
+/**
+ * Wrapper around Cassandra driver session, to automatically handle:
+ * <ul>
+ *  <li>Keyspace and table absence exceptions</li>
+ *  <li>Timeout exceptions</li>
+ *  <li>Batch operations</li>
+ * </ul>
+ */
+public interface CassandraSession extends Closeable {
+    /**
+     * Execute single synchronous operation against Cassandra  database.
+     *
+     * @param assistant execution assistance to perform the main operation logic.
+     * @param <V> type of the result returned from operation.
+     *
+     * @return result of the operation.
+     */
+    public <V> V execute(ExecutionAssistant<V> assistant);
+
+    /**
+     * Executes batch asynchronous operation against Cassandra database.
+     *
+     * @param assistant execution assistance to perform the main operation logic.
+     * @param data data which should be processed in batch operation.
+     * @param <R> type of the result returned from batch operation.
+     * @param <V> type of the value used in batch operation.
+     *
+     * @return result of the operation.
+     */
+    public <R, V> R execute(BatchExecutionAssistant<R, V> assistant, Iterable<? extends V> data);
+
+    /**
+     * Executes batch asynchronous operation to load bunch of records
+     * specified by CQL statement from Cassandra database
+     *
+     * @param assistant execution assistance to perform the main operation logic.
+     */
+    public void execute(BatchLoaderAssistant assistant);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java
new file mode 100644
index 0000000..95b8581
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java
@@ -0,0 +1,832 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.BatchStatement;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.exceptions.AlreadyExistsException;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import com.datastax.driver.core.querybuilder.Batch;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.cassandra.common.CassandraHelper;
+import org.apache.ignite.cache.store.cassandra.common.RandomSleeper;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.cache.store.cassandra.session.pool.SessionPool;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+
+/**
+ * Implementation for {@link org.apache.ignite.cache.store.cassandra.session.CassandraSession}.
+ */
+public class CassandraSessionImpl implements CassandraSession {
+    /** Number of CQL query execution attempts. */
+    private static final int CQL_EXECUTION_ATTEMPTS_COUNT = 20;
+
+    /** Min timeout between CQL query execution attempts. */
+    private static final int CQL_EXECUTION_ATTEMPT_MIN_TIMEOUT = 100;
+
+    /** Max timeout between CQL query execution attempts. */
+    private static final int CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT = 500;
+
+    /** Timeout increment for CQL query execution attempts. */
+    private static final int CQL_ATTEMPTS_TIMEOUT_INCREMENT = 100;
+
+    /** Cassandra cluster builder. */
+    private volatile Cluster.Builder builder;
+
+    /** Cassandra driver session. */
+    private volatile Session ses;
+
+    /** Number of references to Cassandra driver session (for multithreaded environment). */
+    private volatile int refCnt = 0;
+
+    /** Storage for the session prepared statements */
+    private static final Map<String, PreparedStatement> sesStatements = new HashMap<>();
+
+    /** Number of records to immediately fetch in CQL statement execution. */
+    private Integer fetchSize;
+
+    /** Consistency level for Cassandra READ operations (select). */
+    private ConsistencyLevel readConsistency;
+
+    /** Consistency level for Cassandra WRITE operations (insert/update/delete). */
+    private ConsistencyLevel writeConsistency;
+
+    /** Logger. */
+    private IgniteLogger log;
+
+    /** Table absence error handlers counter. */
+    private final AtomicInteger tblAbsenceHandlersCnt = new AtomicInteger(-1);
+
+    /** Prepared statement cluster disconnection error handlers counter. */
+    private final AtomicInteger prepStatementHandlersCnt = new AtomicInteger(-1);
+
+    /**
+     * Creates instance of Cassandra driver session wrapper.
+     *
+     * @param builder Builder for Cassandra cluster.
+     * @param fetchSize Number of rows to immediately fetch in CQL statement execution.
+     * @param readConsistency Consistency level for Cassandra READ operations (select).
+     * @param writeConsistency Consistency level for Cassandra WRITE operations (insert/update/delete).
+     * @param log Logger.
+     */
+    public CassandraSessionImpl(Cluster.Builder builder, Integer fetchSize, ConsistencyLevel readConsistency,
+        ConsistencyLevel writeConsistency, IgniteLogger log) {
+        this.builder = builder;
+        this.fetchSize = fetchSize;
+        this.readConsistency = readConsistency;
+        this.writeConsistency = writeConsistency;
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <V> V execute(ExecutionAssistant<V> assistant) {
+        int attempt = 0;
+        Throwable error = null;
+        String errorMsg = "Failed to execute Cassandra CQL statement: " + assistant.getStatement();
+
+        RandomSleeper sleeper = newSleeper();
+
+        incrementSessionRefs();
+
+        try {
+            while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+                error = null;
+
+                if (attempt != 0) {
+                    log.warning("Trying " + (attempt + 1) + " attempt to execute Cassandra CQL statement: " +
+                            assistant.getStatement());
+                }
+
+                try {
+                    PreparedStatement preparedSt = prepareStatement(assistant.getStatement(),
+                        assistant.getPersistenceSettings(), assistant.tableExistenceRequired());
+
+                    if (preparedSt == null)
+                        return null;
+
+                    Statement statement = tuneStatementExecutionOptions(assistant.bindStatement(preparedSt));
+                    ResultSet res = session().execute(statement);
+
+                    Row row = res == null || !res.iterator().hasNext() ? null : res.iterator().next();
+
+                    return row == null ? null : assistant.process(row);
+                }
+                catch (Throwable e) {
+                    error = e;
+
+                    if (CassandraHelper.isTableAbsenceError(e)) {
+                        if (!assistant.tableExistenceRequired()) {
+                            log.warning(errorMsg, e);
+                            return null;
+                        }
+
+                        handleTableAbsenceError(assistant.getPersistenceSettings());
+                    }
+                    else if (CassandraHelper.isHostsAvailabilityError(e))
+                        handleHostsAvailabilityError(e, attempt, errorMsg);
+                    else if (CassandraHelper.isPreparedStatementClusterError(e))
+                        handlePreparedStatementClusterError(e);
+                    else
+                        // For an error which we don't know how to handle, we will not try next attempts and terminate.
+                        throw new IgniteException(errorMsg, e);
+                }
+
+                sleeper.sleep();
+
+                attempt++;
+            }
+        }
+        catch (Throwable e) {
+            error = e;
+        }
+        finally {
+            decrementSessionRefs();
+        }
+
+        log.error(errorMsg, error);
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R, V> R execute(BatchExecutionAssistant<R, V> assistant, Iterable<? extends V> data) {
+        if (data == null || !data.iterator().hasNext())
+            return assistant.processedData();
+
+        int attempt = 0;
+        String errorMsg = "Failed to execute Cassandra " + assistant.operationName() + " operation";
+        Throwable error = new IgniteException(errorMsg);
+
+        RandomSleeper sleeper = newSleeper();
+
+        int dataSize = 0;
+
+        incrementSessionRefs();
+
+        try {
+            while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+                if (attempt != 0) {
+                    log.warning("Trying " + (attempt + 1) + " attempt to execute Cassandra batch " +
+                            assistant.operationName() + " operation to process rest " +
+                            (dataSize - assistant.processedCount()) + " of " + dataSize + " elements");
+                }
+
+                //clean errors info before next communication with Cassandra
+                Throwable unknownEx = null;
+                Throwable tblAbsenceEx = null;
+                Throwable hostsAvailEx = null;
+                Throwable prepStatEx = null;
+
+                List<Cache.Entry<Integer, ResultSetFuture>> futResults = new LinkedList<>();
+
+                PreparedStatement preparedSt = prepareStatement(assistant.getStatement(),
+                    assistant.getPersistenceSettings(), assistant.tableExistenceRequired());
+
+                if (preparedSt == null)
+                    return null;
+
+                int seqNum = 0;
+
+                for (V obj : data) {
+                    if (!assistant.alreadyProcessed(seqNum)) {
+                        try {
+                            Statement statement = tuneStatementExecutionOptions(assistant.bindStatement(preparedSt, obj));
+                            ResultSetFuture fut = session().executeAsync(statement);
+                            futResults.add(new CacheEntryImpl<>(seqNum, fut));
+                        }
+                        catch (Throwable e) {
+                            if (CassandraHelper.isTableAbsenceError(e)) {
+                                // If there are table absence error and it is not required for the operation we can return.
+                                if (!assistant.tableExistenceRequired())
+                                    return assistant.processedData();
+
+                                tblAbsenceEx = e;
+                                handleTableAbsenceError(assistant.getPersistenceSettings());
+                            }
+                            else if (CassandraHelper.isHostsAvailabilityError(e)) {
+                                hostsAvailEx = e;
+
+                                // Handle host availability only once.
+                                if (hostsAvailEx == null)
+                                    handleHostsAvailabilityError(e, attempt, errorMsg);
+                            }
+                            else if (CassandraHelper.isPreparedStatementClusterError(e)) {
+                                prepStatEx = e;
+                                handlePreparedStatementClusterError(e);
+                            }
+                            else
+                                unknownEx = e;
+                        }
+                    }
+
+                    seqNum++;
+                }
+
+                dataSize = seqNum;
+
+                // For an error which we don't know how to handle, we will not try next attempts and terminate.
+                if (unknownEx != null)
+                    throw new IgniteException(errorMsg, unknownEx);
+
+                // Remembering any of last errors.
+                if (tblAbsenceEx != null)
+                    error = tblAbsenceEx;
+                else if (hostsAvailEx != null)
+                    error = hostsAvailEx;
+                else if (prepStatEx != null)
+                    error = prepStatEx;
+
+                // Clean errors info before next communication with Cassandra.
+                unknownEx = null;
+                tblAbsenceEx = null;
+                hostsAvailEx = null;
+                prepStatEx = null;
+
+                for (Cache.Entry<Integer, ResultSetFuture> futureResult : futResults) {
+                    try {
+                        ResultSet resSet = futureResult.getValue().getUninterruptibly();
+                        Row row = resSet != null && resSet.iterator().hasNext() ? resSet.iterator().next() : null;
+
+                        if (row != null)
+                            assistant.process(row, futureResult.getKey());
+                    }
+                    catch (Throwable e) {
+                        if (CassandraHelper.isTableAbsenceError(e))
+                            tblAbsenceEx = e;
+                        else if (CassandraHelper.isHostsAvailabilityError(e))
+                            hostsAvailEx = e;
+                        else if (CassandraHelper.isPreparedStatementClusterError(e))
+                            prepStatEx = e;
+                        else
+                            unknownEx = e;
+                    }
+                }
+
+                // For an error which we don't know how to handle, we will not try next attempts and terminate.
+                if (unknownEx != null)
+                    throw new IgniteException(errorMsg, unknownEx);
+
+                // If there are no errors occurred it means that operation successfully completed and we can return.
+                if (tblAbsenceEx == null && hostsAvailEx == null && prepStatEx == null)
+                    return assistant.processedData();
+
+                if (tblAbsenceEx != null) {
+                    // If there are table absence error and it is not required for the operation we can return.
+                    if (!assistant.tableExistenceRequired())
+                        return assistant.processedData();
+
+                    error = tblAbsenceEx;
+                    handleTableAbsenceError(assistant.getPersistenceSettings());
+                }
+
+                if (hostsAvailEx != null) {
+                    error = hostsAvailEx;
+                    handleHostsAvailabilityError(hostsAvailEx, attempt, errorMsg);
+                }
+
+                if (prepStatEx != null) {
+                    error = prepStatEx;
+                    handlePreparedStatementClusterError(prepStatEx);
+                }
+
+                sleeper.sleep();
+
+                attempt++;
+            }
+        }
+        catch (Throwable e) {
+            error = e;
+        }
+        finally {
+            decrementSessionRefs();
+        }
+
+        errorMsg = "Failed to process " + (dataSize - assistant.processedCount()) +
+            " of " + dataSize + " elements, during " + assistant.operationName() +
+            " operation with Cassandra";
+
+        log.error(errorMsg, error);
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(BatchLoaderAssistant assistant) {
+        int attempt = 0;
+        String errorMsg = "Failed to execute Cassandra " + assistant.operationName() + " operation";
+        Throwable error = new IgniteException(errorMsg);
+
+        RandomSleeper sleeper = newSleeper();
+
+        incrementSessionRefs();
+
+        try {
+            while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+                if (attempt != 0)
+                    log.warning("Trying " + (attempt + 1) + " attempt to load Ignite cache");
+
+                Statement statement = tuneStatementExecutionOptions(assistant.getStatement());
+
+                try {
+                    ResultSetFuture fut = session().executeAsync(statement);
+                    ResultSet resSet = fut.getUninterruptibly();
+
+                    if (resSet == null || !resSet.iterator().hasNext())
+                        return;
+
+                    for (Row row : resSet)
+                        assistant.process(row);
+
+                    return;
+                }
+                catch (Throwable e) {
+                    error = e;
+
+                    if (CassandraHelper.isTableAbsenceError(e))
+                        return;
+                    else if (CassandraHelper.isHostsAvailabilityError(e))
+                        handleHostsAvailabilityError(e, attempt, errorMsg);
+                    else if (CassandraHelper.isPreparedStatementClusterError(e))
+                        handlePreparedStatementClusterError(e);
+                    else
+                        // For an error which we don't know how to handle, we will not try next attempts and terminate.
+                        throw new IgniteException(errorMsg, e);
+                }
+
+                sleeper.sleep();
+
+                attempt++;
+            }
+        }
+        catch (Throwable e) {
+            error = e;
+        }
+        finally {
+            decrementSessionRefs();
+        }
+
+        log.error(errorMsg, error);
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void close() throws IOException {
+        if (decrementSessionRefs() == 0 && ses != null) {
+            SessionPool.put(this, ses);
+            ses = null;
+        }
+    }
+
+    /**
+     * Recreates Cassandra driver session.
+     */
+    private synchronized void refresh() {
+        //make sure that session removed from the pool
+        SessionPool.get(this);
+
+        //closing and reopening session
+        CassandraHelper.closeSession(ses);
+        ses = null;
+        session();
+
+        synchronized (sesStatements) {
+            sesStatements.clear();
+        }
+    }
+
+    /**
+     * @return Cassandra driver session.
+     */
+    private synchronized Session session() {
+        if (ses != null)
+            return ses;
+
+        ses = SessionPool.get(this);
+
+        if (ses != null)
+            return ses;
+
+        synchronized (sesStatements) {
+            sesStatements.clear();
+        }
+
+        try {
+            return ses = builder.build().connect();
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to establish session with Cassandra database", e);
+        }
+    }
+
+    /**
+     * Increments number of references to Cassandra driver session (required for multithreaded environment).
+     */
+    private synchronized void incrementSessionRefs() {
+        refCnt++;
+    }
+
+    /**
+     * Decrements number of references to Cassandra driver session (required for multithreaded environment).
+     */
+    private synchronized int decrementSessionRefs() {
+        if (refCnt != 0)
+            refCnt--;
+
+        return refCnt;
+    }
+
+    /**
+     * Prepares CQL statement using current Cassandra driver session.
+     *
+     * @param statement CQL statement.
+     * @param settings Persistence settings.
+     * @param tblExistenceRequired Flag indicating if table existence is required for the statement.
+     * @return Prepared statement.
+     */
+    private PreparedStatement prepareStatement(String statement, KeyValuePersistenceSettings settings,
+        boolean tblExistenceRequired) {
+
+        int attempt = 0;
+        Throwable error = null;
+        String errorMsg = "Failed to prepare Cassandra CQL statement: " + statement;
+
+        RandomSleeper sleeper = newSleeper();
+
+        incrementSessionRefs();
+
+        try {
+            synchronized (sesStatements) {
+                if (sesStatements.containsKey(statement))
+                    return sesStatements.get(statement);
+            }
+
+            while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+                try {
+                    PreparedStatement prepStatement = session().prepare(statement);
+
+                    synchronized (sesStatements) {
+                        sesStatements.put(statement, prepStatement);
+                    }
+
+                    return prepStatement;
+                }
+                catch (Throwable e) {
+                    if (CassandraHelper.isTableAbsenceError(e)) {
+                        if (!tblExistenceRequired)
+                            return null;
+
+                        handleTableAbsenceError(settings);
+                    }
+                    else if (CassandraHelper.isHostsAvailabilityError(e))
+                        handleHostsAvailabilityError(e, attempt, errorMsg);
+                    else
+                        throw new IgniteException(errorMsg, e);
+
+                    error = e;
+                }
+
+                sleeper.sleep();
+
+                attempt++;
+            }
+        }
+        finally {
+            decrementSessionRefs();
+        }
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /**
+     * Creates Cassandra keyspace.
+     *
+     * @param settings Persistence settings.
+     */
+    private void createKeyspace(KeyValuePersistenceSettings settings) {
+        int attempt = 0;
+        Throwable error = null;
+        String errorMsg = "Failed to create Cassandra keyspace '" + settings.getKeyspace() + "'";
+
+        while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+            try {
+                log.info("-----------------------------------------------------------------------");
+                log.info("Creating Cassandra keyspace '" + settings.getKeyspace() + "'");
+                log.info("-----------------------------------------------------------------------\n\n" +
+                    settings.getKeyspaceDDLStatement() + "\n");
+                log.info("-----------------------------------------------------------------------");
+                session().execute(settings.getKeyspaceDDLStatement());
+                log.info("Cassandra keyspace '" + settings.getKeyspace() + "' was successfully created");
+                return;
+            }
+            catch (AlreadyExistsException ignored) {
+                log.info("Cassandra keyspace '" + settings.getKeyspace() + "' already exist");
+                return;
+            }
+            catch (Throwable e) {
+                if (!CassandraHelper.isHostsAvailabilityError(e))
+                    throw new IgniteException(errorMsg, e);
+
+                handleHostsAvailabilityError(e, attempt, errorMsg);
+
+                error = e;
+            }
+
+            attempt++;
+        }
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /**
+     * Creates Cassandra table.
+     *
+     * @param settings Persistence settings.
+     */
+    private void createTable(KeyValuePersistenceSettings settings) {
+        int attempt = 0;
+        Throwable error = null;
+        String errorMsg = "Failed to create Cassandra table '" + settings.getTableFullName() + "'";
+
+        while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+            try {
+                log.info("-----------------------------------------------------------------------");
+                log.info("Creating Cassandra table '" + settings.getTableFullName() + "'");
+                log.info("-----------------------------------------------------------------------\n\n" +
+                    settings.getTableDDLStatement() + "\n");
+                log.info("-----------------------------------------------------------------------");
+                session().execute(settings.getTableDDLStatement());
+                log.info("Cassandra table '" + settings.getTableFullName() + "' was successfully created");
+                return;
+            }
+            catch (AlreadyExistsException ignored) {
+                log.info("Cassandra table '" + settings.getTableFullName() + "' already exist");
+                return;
+            }
+            catch (Throwable e) {
+                if (!CassandraHelper.isHostsAvailabilityError(e) && !CassandraHelper.isKeyspaceAbsenceError(e))
+                    throw new IgniteException(errorMsg, e);
+
+                if (CassandraHelper.isKeyspaceAbsenceError(e)) {
+                    log.warning("Failed to create Cassandra table '" + settings.getTableFullName() +
+                        "' cause appropriate keyspace doesn't exist", e);
+                    createKeyspace(settings);
+                }
+                else if (CassandraHelper.isHostsAvailabilityError(e))
+                    handleHostsAvailabilityError(e, attempt, errorMsg);
+
+                error = e;
+            }
+
+            attempt++;
+        }
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /**
+     * Creates Cassandra table indexes.
+     *
+     * @param settings Persistence settings.
+     */
+    private void createTableIndexes(KeyValuePersistenceSettings settings) {
+        if (settings.getIndexDDLStatements() == null || settings.getIndexDDLStatements().isEmpty())
+            return;
+
+        int attempt = 0;
+        Throwable error = null;
+        String errorMsg = "Failed to create indexes for Cassandra table " + settings.getTableFullName();
+
+        while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+            try {
+                log.info("Creating indexes for Cassandra table '" + settings.getTableFullName() + "'");
+
+                for (String statement : settings.getIndexDDLStatements()) {
+                    try {
+                        session().execute(statement);
+                    }
+                    catch (AlreadyExistsException ignored) {
+                    }
+                    catch (Throwable e) {
+                        if (!(e instanceof InvalidQueryException) || !e.getMessage().equals("Index already exists"))
+                            throw new IgniteException(errorMsg, e);
+                    }
+                }
+
+                log.info("Indexes for Cassandra table '" + settings.getTableFullName() + "' were successfully created");
+
+                return;
+            }
+            catch (Throwable e) {
+                if (CassandraHelper.isHostsAvailabilityError(e))
+                    handleHostsAvailabilityError(e, attempt, errorMsg);
+                else if (CassandraHelper.isTableAbsenceError(e))
+                    createTable(settings);
+                else
+                    throw new IgniteException(errorMsg, e);
+
+                error = e;
+            }
+
+            attempt++;
+        }
+
+        throw new IgniteException(errorMsg, error);
+    }
+
+    /**
+     * Tunes CQL statement execution options (consistency level, fetch option and etc.).
+     *
+     * @param statement Statement.
+     * @return Modified statement.
+     */
+    private Statement tuneStatementExecutionOptions(Statement statement) {
+        String qry = "";
+
+        if (statement instanceof BoundStatement)
+            qry = ((BoundStatement)statement).preparedStatement().getQueryString().trim().toLowerCase();
+        else if (statement instanceof PreparedStatement)
+            qry = ((PreparedStatement)statement).getQueryString().trim().toLowerCase();
+
+        boolean readStatement = qry.startsWith("select");
+        boolean writeStatement = statement instanceof Batch || statement instanceof BatchStatement ||
+            qry.startsWith("insert") || qry.startsWith("delete") || qry.startsWith("update");
+
+        if (readStatement && readConsistency != null)
+            statement.setConsistencyLevel(readConsistency);
+
+        if (writeStatement && writeConsistency != null)
+            statement.setConsistencyLevel(writeConsistency);
+
+        if (fetchSize != null)
+            statement.setFetchSize(fetchSize);
+
+        return statement;
+    }
+
+    /**
+     * Handles situation when Cassandra table doesn't exist.
+     *
+     * @param settings Persistence settings.
+     */
+    private void handleTableAbsenceError(KeyValuePersistenceSettings settings) {
+        int hndNum = tblAbsenceHandlersCnt.incrementAndGet();
+
+        try {
+            synchronized (tblAbsenceHandlersCnt) {
+                // Oooops... I am not the first thread who tried to handle table absence problem.
+                if (hndNum != 0) {
+                    log.warning("Table " + settings.getTableFullName() + " absence problem detected. " +
+                            "Another thread already fixed it.");
+                    return;
+                }
+
+                log.warning("Table " + settings.getTableFullName() + " absence problem detected. " +
+                        "Trying to create table.");
+
+                IgniteException error = new IgniteException("Failed to create Cassandra table " + settings.getTableFullName());
+
+                int attempt = 0;
+
+                while (error != null && attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+                    error = null;
+
+                    try {
+                        createKeyspace(settings);
+                        createTable(settings);
+                        createTableIndexes(settings);
+                    }
+                    catch (Throwable e) {
+                        if (CassandraHelper.isHostsAvailabilityError(e))
+                            handleHostsAvailabilityError(e, attempt, null);
+                        else
+                            throw new IgniteException("Failed to create Cassandra table " + settings.getTableFullName(), e);
+
+                        error = (e instanceof IgniteException) ? (IgniteException)e : new IgniteException(e);
+                    }
+
+                    attempt++;
+                }
+
+                if (error != null)
+                    throw error;
+            }
+        }
+        finally {
+            if (hndNum == 0)
+                tblAbsenceHandlersCnt.set(-1);
+        }
+    }
+
+    /**
+     * Handles situation when prepared statement execution failed cause session to the cluster was released.
+     *
+     */
+    private void handlePreparedStatementClusterError(Throwable e) {
+        int hndNum = prepStatementHandlersCnt.incrementAndGet();
+
+        try {
+            synchronized (prepStatementHandlersCnt) {
+                // Oooops... I am not the first thread who tried to handle prepared statement problem.
+                if (hndNum != 0) {
+                    log.warning("Prepared statement cluster error detected, another thread already fixed the problem", e);
+                    return;
+                }
+
+                log.warning("Prepared statement cluster error detected, refreshing Cassandra session", e);
+
+                refresh();
+
+                log.warning("Cassandra session refreshed");
+            }
+        }
+        finally {
+            if (hndNum == 0)
+                prepStatementHandlersCnt.set(-1);
+        }
+    }
+
+    /**
+     * Handles situation when Cassandra host which is responsible for CQL query execution became unavailable.
+     *
+     * @param e Exception to handle.
+     * @param attempt Number of attempts.
+     * @param msg Error message.
+     * @return {@code true} if host unavailability was successfully handled.
+     */
+    private boolean handleHostsAvailabilityError(Throwable e, int attempt, String msg) {
+        if (attempt >= CQL_EXECUTION_ATTEMPTS_COUNT) {
+            log.error("Host availability problem detected. " +
+                    "Number of CQL execution attempts reached maximum " + CQL_EXECUTION_ATTEMPTS_COUNT +
+                    ", exception will be thrown to upper execution layer.", e);
+            throw msg == null ? new IgniteException(e) : new IgniteException(msg, e);
+        }
+
+        if (attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 4  ||
+            attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 2  ||
+            attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 2 + CQL_EXECUTION_ATTEMPTS_COUNT / 4  ||
+            attempt == CQL_EXECUTION_ATTEMPTS_COUNT - 1) {
+            log.warning("Host availability problem detected, CQL execution attempt  " + (attempt + 1) + ", " +
+                    "refreshing Cassandra session", e);
+
+            refresh();
+
+            log.warning("Cassandra session refreshed");
+
+            return true;
+        }
+
+        log.warning("Host availability problem detected, CQL execution attempt " + (attempt + 1) + ", " +
+                "sleeping extra " + CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT + " milliseconds", e);
+
+        try {
+            Thread.sleep(CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT);
+        }
+        catch (InterruptedException ignored) {
+        }
+
+        log.warning("Sleep completed");
+
+        return false;
+    }
+
+    /**
+     * @return New random sleeper.
+     */
+    private RandomSleeper newSleeper() {
+        return new RandomSleeper(CQL_EXECUTION_ATTEMPT_MIN_TIMEOUT,
+                CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT,
+                CQL_ATTEMPTS_TIMEOUT_INCREMENT, log);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java
new file mode 100644
index 0000000..867f58d
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Provides information for single operations (load, delete, write) of Ignite cache
+ * backed by {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}.
+ *
+ * @param <R> type of the result returned from operation.
+ */
+public interface ExecutionAssistant<R> {
+    /**
+     * Indicates if Cassandra table existence is required for operation.
+     *
+     * @return true if table existence required.
+     */
+    public boolean tableExistenceRequired();
+
+    /**
+     * Returns CQL statement to be used for operation.
+     *
+     * @return CQL statement.
+     */
+    public String getStatement();
+
+    /**
+     * Binds prepared statement.
+     *
+     * @param statement prepared statement.
+     *
+     * @return bound statement.
+     */
+    public BoundStatement bindStatement(PreparedStatement statement);
+
+    /**
+     * Persistence settings to use for operation.
+     *
+     * @return persistence settings.
+     */
+    public KeyValuePersistenceSettings getPersistenceSettings();
+
+    /**
+     * Returns operation name.
+     *
+     * @return operation name.
+     */
+    public String operationName();
+
+    /**
+     * Processes Cassandra database table row returned by specified CQL statement.
+     *
+     * @param row Cassandra database table row.
+     *
+     * @return result of the operation.
+     */
+    public R process(Row row);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
new file mode 100644
index 0000000..e80583a
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.Row;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Implementation of the {@link org.apache.ignite.cache.store.cassandra.session.BatchExecutionAssistant}.
+ *
+ * @param <R> Type of the result returned from batch operation
+ * @param <V> Type of the value used in batch operation
+ */
+public abstract class GenericBatchExecutionAssistant<R, V> implements BatchExecutionAssistant<R, V> {
+    /** Identifiers of already processed objects. */
+    private Set<Integer> processed = new HashSet<>();
+
+    /** {@inheritDoc} */
+    @Override public void process(Row row, int seqNum) {
+        if (processed.contains(seqNum))
+            return;
+
+        process(row);
+
+        processed.add(seqNum);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean alreadyProcessed(int seqNum) {
+        return processed.contains(seqNum);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int processedCount() {
+        return processed.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R processedData() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean tableExistenceRequired() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    protected void process(Row row) {
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java
new file mode 100644
index 0000000..d3ace7d
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.Statement;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
+import org.apache.ignite.lang.IgniteBiInClosure;
+
+/**
+ * Worker for load cache using custom user query.
+ *
+ * @param <K> Key type.
+ * @param <V> Value type.
+ */
+public class LoadCacheCustomQueryWorker<K, V> implements Callable<Void> {
+    /** Cassandra session to execute CQL query */
+    private final CassandraSession ses;
+
+    /** User query. */
+    private final String qry;
+
+    /** Persistence controller */
+    private final PersistenceController ctrl;
+
+    /** Logger */
+    private final IgniteLogger log;
+
+    /** Closure for loaded values. */
+    private final IgniteBiInClosure<K, V> clo;
+
+    /**
+     * @param clo Closure for loaded values.
+     */
+    public LoadCacheCustomQueryWorker(CassandraSession ses, String qry, PersistenceController ctrl,
+        IgniteLogger log, IgniteBiInClosure<K, V> clo) {
+        this.ses = ses;
+        this.qry = qry.trim().endsWith(";") ? qry : qry + ";";
+        this.ctrl = ctrl;
+        this.log = log;
+        this.clo = clo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void call() throws Exception {
+        ses.execute(new BatchLoaderAssistant() {
+            /** {@inheritDoc} */
+            @Override public String operationName() {
+                return "loadCache";
+            }
+
+            /** {@inheritDoc} */
+            @Override public Statement getStatement() {
+                return new SimpleStatement(qry);
+            }
+
+            /** {@inheritDoc} */
+            @Override public void process(Row row) {
+                K key;
+                V val;
+
+                try {
+                    key = (K)ctrl.buildKeyObject(row);
+                }
+                catch (Throwable e) {
+                    log.error("Failed to build Ignite key object from provided Cassandra row", e);
+
+                    throw new IgniteException("Failed to build Ignite key object from provided Cassandra row", e);
+                }
+
+                try {
+                    val = (V)ctrl.buildValueObject(row);
+                }
+                catch (Throwable e) {
+                    log.error("Failed to build Ignite value object from provided Cassandra row", e);
+
+                    throw new IgniteException("Failed to build Ignite value object from provided Cassandra row", e);
+                }
+
+                clo.apply(key, val);
+            }
+        });
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java
new file mode 100644
index 0000000..fc4a907
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session.pool;
+
+import com.datastax.driver.core.Session;
+import java.lang.Thread.State;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl;
+
+/**
+ * Cassandra driver sessions pool.
+ */
+public class SessionPool {
+    /**
+     * Monitors session pool and closes unused session.
+     */
+    private static class SessionMonitor extends Thread {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                while (true) {
+                    try {
+                        Thread.sleep(SLEEP_TIMEOUT);
+                    }
+                    catch (InterruptedException ignored) {
+                        return;
+                    }
+
+                    List<Map.Entry<CassandraSessionImpl, SessionWrapper>> expiredSessions = new LinkedList<>();
+
+                    int sessionsCnt;
+
+                    synchronized (sessions) {
+                        sessionsCnt = sessions.size();
+
+                        for (Map.Entry<CassandraSessionImpl, SessionWrapper> entry : sessions.entrySet()) {
+                            if (entry.getValue().expired())
+                                expiredSessions.add(entry);
+                        }
+
+                        for (Map.Entry<CassandraSessionImpl, SessionWrapper> entry : expiredSessions)
+                            sessions.remove(entry.getKey());
+                    }
+
+                    for (Map.Entry<CassandraSessionImpl, SessionWrapper> entry : expiredSessions)
+                        entry.getValue().release();
+
+                    // all sessions in the pool expired, thus we don't need additional thread to manage sessions in the pool
+                    if (sessionsCnt == expiredSessions.size())
+                        return;
+                }
+            }
+            finally {
+                release();
+            }
+        }
+    }
+
+    /** Sessions monitor sleep timeout. */
+    private static final long SLEEP_TIMEOUT = 60000; // 1 minute.
+
+    /** Sessions which were returned to pool. */
+    private static final Map<CassandraSessionImpl, SessionWrapper> sessions = new HashMap<>();
+
+    /** Singleton instance. */
+    private static SessionMonitor monitorSingleton;
+
+    static {
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override public void run() {
+                release();
+            }
+        });
+    }
+
+    /**
+     * Returns Cassandra driver session to sessions pool.
+     *
+     * @param cassandraSes Session wrapper.
+     * @param driverSes Driver session.
+     */
+    public static void put(CassandraSessionImpl cassandraSes, Session driverSes) {
+        if (cassandraSes == null || driverSes == null)
+            return;
+
+        SessionWrapper old;
+
+        synchronized (sessions) {
+            old = sessions.put(cassandraSes, new SessionWrapper(driverSes));
+
+            if (monitorSingleton == null || State.TERMINATED.equals(monitorSingleton.getState())) {
+                monitorSingleton = new SessionMonitor();
+                monitorSingleton.setDaemon(true);
+                monitorSingleton.setName("Cassandra-sessions-pool");
+                monitorSingleton.start();
+            }
+        }
+
+        if (old != null)
+            old.release();
+    }
+
+    /**
+     * Extracts Cassandra driver session from pool.
+     *
+     * @param cassandraSes Session wrapper.
+     * @return Cassandra driver session.
+     */
+    public static Session get(CassandraSessionImpl cassandraSes) {
+        if (cassandraSes == null)
+            return null;
+
+        SessionWrapper wrapper;
+
+        synchronized (sessions) {
+            wrapper = sessions.remove(cassandraSes);
+        }
+
+        return wrapper == null ? null : wrapper.driverSession();
+    }
+
+    /**
+     * Releases all session from pool and closes all their connections to Cassandra database.
+     */
+    public static void release() {
+        Collection<SessionWrapper> wrappers;
+
+        synchronized (sessions) {
+            try {
+                if (sessions.size() == 0)
+                    return;
+
+                wrappers = new LinkedList<>();
+
+                for (SessionWrapper wrapper : sessions.values())
+                    wrappers.add(wrapper);
+
+                sessions.clear();
+            }
+            finally {
+                if (!(Thread.currentThread() instanceof SessionMonitor) && monitorSingleton != null) {
+                    try {
+                        monitorSingleton.interrupt();
+                    }
+                    catch (Throwable ignored) {
+                    }
+                }
+            }
+        }
+
+        for (SessionWrapper wrapper : wrappers)
+            wrapper.release();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java
new file mode 100644
index 0000000..7c5722b
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionWrapper.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.session.pool;
+
+import com.datastax.driver.core.Session;
+import org.apache.ignite.cache.store.cassandra.common.CassandraHelper;
+
+/**
+ * Wrapper for Cassandra driver session, responsible for monitoring session expiration and its closing.
+ */
+public class SessionWrapper {
+    /** Expiration timeout for Cassandra driver session. */
+    public static final long DFLT_EXPIRATION_TIMEOUT = 300000;  // 5 minutes.
+
+    /** Cassandra driver session. */
+    private Session ses;
+
+    /** Wrapper creation time.  */
+    private long time;
+
+    /**
+     * Creates instance of Cassandra driver session wrapper.
+     *
+     * @param ses Cassandra driver session.
+     */
+    public SessionWrapper(Session ses) {
+        this.ses = ses;
+        this.time = System.currentTimeMillis();
+    }
+
+    /**
+     * Checks if Cassandra driver session expired.
+     *
+     * @return true if session expired.
+     */
+    public boolean expired() {
+        return System.currentTimeMillis() - time > DFLT_EXPIRATION_TIMEOUT;
+    }
+
+    /**
+     * Returns wrapped Cassandra driver session.
+     *
+     * @return Cassandra driver session.
+     */
+    public Session driverSession() {
+        return ses;
+    }
+
+    /**
+     * Closes wrapped Cassandra driver session
+     */
+    public void release() {
+        CassandraHelper.closeSession(ses);
+        ses = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java
new file mode 100644
index 0000000..4f40478
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.utils;
+
+import java.io.File;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Generates Cassandra DDL statements from persistence descriptor xml file.
+ */
+public class DDLGenerator {
+    /**
+     * DDLGenerator entry point.
+     *
+     * @param args Arguments for DDLGenerator.
+     */
+    public static void main(String[] args) {
+        if (args == null || args.length == 0)
+            return;
+
+        for (String arg : args) {
+            File file = new File(arg);
+            if (!file.isFile()) {
+                System.out.println("-------------------------------------------------------------");
+                System.out.println("Incorrect file specified: " + arg);
+                System.out.println("-------------------------------------------------------------");
+                continue;
+            }
+
+            try {
+                KeyValuePersistenceSettings settings = new KeyValuePersistenceSettings(file);
+                System.out.println("-------------------------------------------------------------");
+                System.out.println("DDL for keyspace/table from file: " + arg);
+                System.out.println("-------------------------------------------------------------");
+                System.out.println();
+                System.out.println(settings.getKeyspaceDDLStatement());
+                System.out.println();
+                System.out.println(settings.getTableDDLStatement());
+                System.out.println();
+            }
+            catch (Throwable e) {
+                System.out.println("-------------------------------------------------------------");
+                System.out.println("Incorrect file specified: " + arg);
+                System.out.println("-------------------------------------------------------------");
+                e.printStackTrace();
+            }
+        }
+    }
+}


[25/50] ignite git commit: IGNITE-1371 Implemented Cassandra cache store.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
new file mode 100644
index 0000000..1ecb28f
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
@@ -0,0 +1,550 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.datasource;
+
+import com.datastax.driver.core.AuthProvider;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.NettyOptions;
+import com.datastax.driver.core.PoolingOptions;
+import com.datastax.driver.core.ProtocolOptions;
+import com.datastax.driver.core.ProtocolVersion;
+import com.datastax.driver.core.SSLOptions;
+import com.datastax.driver.core.SocketOptions;
+import com.datastax.driver.core.policies.AddressTranslator;
+import com.datastax.driver.core.policies.LoadBalancingPolicy;
+import com.datastax.driver.core.policies.ReconnectionPolicy;
+import com.datastax.driver.core.policies.RetryPolicy;
+import com.datastax.driver.core.policies.SpeculativeExecutionPolicy;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
+import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl;
+
+/**
+ * Data source abstraction to specify configuration of the Cassandra session to be used.
+ */
+public class DataSource {
+    /** Number of rows to immediately fetch in CQL statement execution. */
+    private Integer fetchSize;
+
+    /** Consistency level for READ operations. */
+    private ConsistencyLevel readConsistency;
+
+    /** Consistency level for WRITE operations. */
+    private ConsistencyLevel writeConsistency;
+
+    /** Username to use for authentication. */
+    private String user;
+
+    /** Password to use for authentication. */
+    private String pwd;
+
+    /** Port to use for Cassandra connection. */
+    private Integer port;
+
+    /** List of contact points to connect to Cassandra cluster. */
+    private List<InetAddress> contactPoints;
+
+    /** List of contact points with ports to connect to Cassandra cluster. */
+    private List<InetSocketAddress> contactPointsWithPorts;
+
+    /** Maximum time to wait for schema agreement before returning from a DDL query. */
+    private Integer maxSchemaAgreementWaitSeconds;
+
+    /** The native protocol version to use. */
+    private Integer protoVer;
+
+    /** Compression to use for the transport. */
+    private String compression;
+
+    /** Use SSL for communications with Cassandra. */
+    private Boolean useSSL;
+
+    /** Enables metrics collection. */
+    private Boolean collectMetrix;
+
+    /** Enables JMX reporting of the metrics. */
+    private Boolean jmxReporting;
+
+    /** Credentials to use for authentication. */
+    private Credentials creds;
+
+    /** Load balancing policy to use. */
+    private LoadBalancingPolicy loadBalancingPlc;
+
+    /** Reconnection policy to use. */
+    private ReconnectionPolicy reconnectionPlc;
+
+    /** Retry policy to use. */
+    private RetryPolicy retryPlc;
+
+    /** Address translator to use. */
+    private AddressTranslator addrTranslator;
+
+    /** Speculative execution policy to use. */
+    private SpeculativeExecutionPolicy speculativeExecutionPlc;
+
+    /** Authentication provider to use. */
+    private AuthProvider authProvider;
+
+    /** SSL options to use. */
+    private SSLOptions sslOptions;
+
+    /** Connection pooling options to use. */
+    private PoolingOptions poolingOptions;
+
+    /** Socket options to use. */
+    private SocketOptions sockOptions;
+
+    /** Netty options to use for connection. */
+    private NettyOptions nettyOptions;
+
+    /** Cassandra session wrapper instance. */
+    private volatile CassandraSession ses;
+
+    /**
+     * Sets user name to use for authentication.
+     *
+     * @param user user name
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setUser(String user) {
+        this.user = user;
+
+        invalidate();
+    }
+
+    /**
+     * Sets password to use for authentication.
+     *
+     * @param pwd password
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setPassword(String pwd) {
+        this.pwd = pwd;
+
+        invalidate();
+    }
+
+    /**
+     * Sets port to use for Cassandra connection.
+     *
+     * @param port port
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setPort(int port) {
+        this.port = port;
+
+        invalidate();
+    }
+
+    /**
+     * Sets list of contact points to connect to Cassandra cluster.
+     *
+     * @param points contact points
+     */
+    public void setContactPoints(String... points) {
+        if (points == null || points.length == 0)
+            return;
+
+        for (String point : points) {
+            if (point.contains(":")) {
+                if (contactPointsWithPorts == null)
+                    contactPointsWithPorts = new LinkedList<>();
+
+                String[] chunks = point.split(":");
+
+                try {
+                    contactPointsWithPorts.add(InetSocketAddress.createUnresolved(chunks[0].trim(), Integer.parseInt(chunks[1].trim())));
+                }
+                catch (Throwable e) {
+                    throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
+                }
+            }
+            else {
+                if (contactPoints == null)
+                    contactPoints = new LinkedList<>();
+
+                try {
+                    contactPoints.add(InetAddress.getByName(point));
+                }
+                catch (Throwable e) {
+                    throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
+                }
+            }
+        }
+
+        invalidate();
+    }
+
+    /** Sets maximum time to wait for schema agreement before returning from a DDL query. */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setMaxSchemaAgreementWaitSeconds(int seconds) {
+        maxSchemaAgreementWaitSeconds = seconds;
+
+        invalidate();
+    }
+
+    /**
+     * Sets the native protocol version to use.
+     *
+     * @param ver version number
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setProtocolVersion(int ver) {
+        protoVer = ver;
+
+        invalidate();
+    }
+
+    /**
+     * Sets compression algorithm to use for the transport.
+     *
+     * @param compression Compression algorithm.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setCompression(String compression) {
+        this.compression = compression == null || compression.trim().isEmpty() ? null : compression.trim();
+
+        try {
+            if (this.compression != null)
+                ProtocolOptions.Compression.valueOf(this.compression);
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Incorrect compression '" + compression + "' specified for Cassandra connection", e);
+        }
+
+        invalidate();
+    }
+
+    /**
+     * Enables SSL for communications with Cassandra.
+     *
+     * @param use Flag to enable/disable SSL.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setUseSSL(boolean use) {
+        useSSL = use;
+
+        invalidate();
+    }
+
+    /**
+     * Enables metrics collection.
+     *
+     * @param collect Flag to enable/disable metrics collection.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setCollectMetrix(boolean collect) {
+        collectMetrix = collect;
+
+        invalidate();
+    }
+
+    /**
+     * Enables JMX reporting of the metrics.
+     *
+     * @param enableReporting Flag to enable/disable JMX reporting.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setJmxReporting(boolean enableReporting) {
+        jmxReporting = enableReporting;
+
+        invalidate();
+    }
+
+    /**
+     * Sets number of rows to immediately fetch in CQL statement execution.
+     *
+     * @param size Number of rows to fetch.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setFetchSize(int size) {
+        fetchSize = size;
+
+        invalidate();
+    }
+
+    /**
+     * Set consistency level for READ operations.
+     *
+     * @param level Consistency level.
+     */
+    public void setReadConsistency(String level) {
+        readConsistency = parseConsistencyLevel(level);
+
+        invalidate();
+    }
+
+    /**
+     * Set consistency level for WRITE operations.
+     *
+     * @param level Consistency level.
+     */
+    public void setWriteConsistency(String level) {
+        writeConsistency = parseConsistencyLevel(level);
+
+        invalidate();
+    }
+
+    /**
+     * Sets credentials to use for authentication.
+     *
+     * @param creds Credentials.
+     */
+    public void setCredentials(Credentials creds) {
+        this.creds = creds;
+
+        invalidate();
+    }
+
+    /**
+     * Sets load balancing policy.
+     *
+     * @param plc Load balancing policy.
+     */
+    public void setLoadBalancingPolicy(LoadBalancingPolicy plc) {
+        this.loadBalancingPlc = plc;
+
+        invalidate();
+    }
+
+    /**
+     * Sets reconnection policy.
+     *
+     * @param plc Reconnection policy.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setReconnectionPolicy(ReconnectionPolicy plc) {
+        this.reconnectionPlc = plc;
+
+        invalidate();
+    }
+
+    /**
+     * Sets retry policy.
+     *
+     * @param plc Retry policy.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setRetryPolicy(RetryPolicy plc) {
+        this.retryPlc = plc;
+
+        invalidate();
+    }
+
+    /**
+     * Sets address translator.
+     *
+     * @param translator Address translator.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setAddressTranslator(AddressTranslator translator) {
+        this.addrTranslator = translator;
+
+        invalidate();
+    }
+
+    /**
+     * Sets speculative execution policy.
+     *
+     * @param plc Speculative execution policy.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setSpeculativeExecutionPolicy(SpeculativeExecutionPolicy plc) {
+        this.speculativeExecutionPlc = plc;
+
+        invalidate();
+    }
+
+    /**
+     * Sets authentication provider.
+     *
+     * @param provider Authentication provider.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setAuthProvider(AuthProvider provider) {
+        this.authProvider = provider;
+
+        invalidate();
+    }
+
+    /**
+     * Sets SSL options.
+     *
+     * @param options SSL options.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setSslOptions(SSLOptions options) {
+        this.sslOptions = options;
+
+        invalidate();
+    }
+
+    /**
+     * Sets pooling options.
+     *
+     * @param options pooling options to use.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setPoolingOptions(PoolingOptions options) {
+        this.poolingOptions = options;
+
+        invalidate();
+    }
+
+    /**
+     * Sets socket options to use.
+     *
+     * @param options Socket options.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setSocketOptions(SocketOptions options) {
+        this.sockOptions = options;
+
+        invalidate();
+    }
+
+    /**
+     * Sets netty options to use.
+     *
+     * @param options netty options.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public void setNettyOptions(NettyOptions options) {
+        this.nettyOptions = options;
+
+        invalidate();
+    }
+
+    /**
+     * Creates Cassandra session wrapper if it wasn't created yet and returns it
+     *
+     * @param log logger
+     * @return Cassandra session wrapper
+     */
+    @SuppressWarnings("deprecation")
+    public synchronized CassandraSession session(IgniteLogger log) {
+        if (ses != null)
+            return ses;
+
+        Cluster.Builder builder = Cluster.builder();
+
+        if (user != null)
+            builder = builder.withCredentials(user, pwd);
+
+        if (port != null)
+            builder = builder.withPort(port);
+
+        if (contactPoints != null)
+            builder = builder.addContactPoints(contactPoints);
+
+        if (contactPointsWithPorts != null)
+            builder = builder.addContactPointsWithPorts(contactPointsWithPorts);
+
+        if (maxSchemaAgreementWaitSeconds != null)
+            builder = builder.withMaxSchemaAgreementWaitSeconds(maxSchemaAgreementWaitSeconds);
+
+        if (protoVer != null)
+            builder = builder.withProtocolVersion(ProtocolVersion.fromInt(protoVer));
+
+        if (compression != null) {
+            try {
+                builder = builder.withCompression(ProtocolOptions.Compression.valueOf(compression.trim().toLowerCase()));
+            }
+            catch (IllegalArgumentException e) {
+                throw new IgniteException("Incorrect compression option '" + compression + "' specified for Cassandra connection", e);
+            }
+        }
+
+        if (useSSL != null && useSSL)
+            builder = builder.withSSL();
+
+        if (sslOptions != null)
+            builder = builder.withSSL(sslOptions);
+
+        if (collectMetrix != null && !collectMetrix)
+            builder = builder.withoutMetrics();
+
+        if (jmxReporting != null && !jmxReporting)
+            builder = builder.withoutJMXReporting();
+
+        if (creds != null)
+            builder = builder.withCredentials(creds.getUser(), creds.getPassword());
+
+        if (loadBalancingPlc != null)
+            builder = builder.withLoadBalancingPolicy(loadBalancingPlc);
+
+        if (reconnectionPlc != null)
+            builder = builder.withReconnectionPolicy(reconnectionPlc);
+
+        if (retryPlc != null)
+            builder = builder.withRetryPolicy(retryPlc);
+
+        if (addrTranslator != null)
+            builder = builder.withAddressTranslator(addrTranslator);
+
+        if (speculativeExecutionPlc != null)
+            builder = builder.withSpeculativeExecutionPolicy(speculativeExecutionPlc);
+
+        if (authProvider != null)
+            builder = builder.withAuthProvider(authProvider);
+
+        if (poolingOptions != null)
+            builder = builder.withPoolingOptions(poolingOptions);
+
+        if (sockOptions != null)
+            builder = builder.withSocketOptions(sockOptions);
+
+        if (nettyOptions != null)
+            builder = builder.withNettyOptions(nettyOptions);
+
+        return ses = new CassandraSessionImpl(builder, fetchSize, readConsistency, writeConsistency, log);
+    }
+
+    /**
+     * Parses consistency level provided as string.
+     *
+     * @param level consistency level string.
+     *
+     * @return consistency level.
+     */
+    private ConsistencyLevel parseConsistencyLevel(String level) {
+        if (level == null)
+            return null;
+
+        try {
+            return ConsistencyLevel.valueOf(level.trim().toUpperCase());
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Incorrect consistency level '" + level + "' specified for Cassandra connection", e);
+        }
+    }
+
+    /**
+     * Invalidates session.
+     */
+    private synchronized void invalidate() {
+        ses = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java
new file mode 100644
index 0000000..9d0710e
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.datasource;
+
+/**
+ * Simple implementation of {@link Credentials} which just uses its constructor to hold user/password values.
+ */
+public class PlainCredentials implements Credentials {
+    /** User name. */
+    private String user;
+
+    /** User password. */
+    private String pwd;
+
+    /**
+     * Creates credentials object.
+     *
+     * @param user User name.
+     * @param pwd User password.
+     */
+    public PlainCredentials(String user, String pwd) {
+        this.user = user;
+        this.pwd = pwd;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getUser() {
+        return user;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getPassword() {
+        return pwd;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java
new file mode 100644
index 0000000..393dbe4
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import java.beans.PropertyDescriptor;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+/**
+ * Stores persistence settings for Ignite cache key
+ */
+public class KeyPersistenceSettings extends PersistenceSettings {
+    /** Partition key XML tag. */
+    private static final String PARTITION_KEY_ELEMENT = "partitionKey";
+
+    /** Cluster key XML tag. */
+    private static final String CLUSTER_KEY_ELEMENT = "clusterKey";
+
+    /** POJO field XML tag. */
+    private static final String FIELD_ELEMENT = "field";
+
+    /** POJO fields. */
+    private List<PojoField> fields = new LinkedList<>();
+
+    /** Partition key fields. */
+    private List<PojoField> partKeyFields = new LinkedList<>();
+
+    /** Cluster key fields. */
+    private List<PojoField> clusterKeyFields = new LinkedList<>();
+
+    /**
+     * Creates key persistence settings object based on it's XML configuration.
+     *
+     * @param el XML element storing key persistence settings
+     */
+    public KeyPersistenceSettings(Element el) {
+        super(el);
+
+        if (!PersistenceStrategy.POJO.equals(getStrategy()))
+            return;
+
+        NodeList keyElem = el.getElementsByTagName(PARTITION_KEY_ELEMENT);
+
+        Element partKeysNode = keyElem != null ? (Element) keyElem.item(0) : null;
+
+        Element clusterKeysNode = el.getElementsByTagName(CLUSTER_KEY_ELEMENT) != null ?
+            (Element)el.getElementsByTagName(CLUSTER_KEY_ELEMENT).item(0) : null;
+
+        if (partKeysNode == null && clusterKeysNode != null) {
+            throw new IllegalArgumentException("It's not allowed to specify cluster key fields mapping, but " +
+                "doesn't specify partition key mappings");
+        }
+
+        partKeyFields = detectFields(partKeysNode, getPartitionKeyDescriptors());
+
+        if (partKeyFields == null || partKeyFields.isEmpty()) {
+            throw new IllegalStateException("Failed to initialize partition key fields for class '" +
+                getJavaClass().getName() + "'");
+        }
+
+        clusterKeyFields = detectFields(clusterKeysNode, getClusterKeyDescriptors(partKeyFields));
+
+        fields = new LinkedList<>();
+        fields.addAll(partKeyFields);
+        fields.addAll(clusterKeyFields);
+
+        checkDuplicates(fields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<PojoField> getFields() {
+        return fields;
+    }
+
+    /**
+     * Returns Cassandra DDL for primary key.
+     *
+     * @return DDL statement.
+     */
+    public String getPrimaryKeyDDL() {
+        StringBuilder partKey = new StringBuilder();
+
+        List<String> cols = getPartitionKeyColumns();
+        for (String column : cols) {
+            if (partKey.length() != 0)
+                partKey.append(", ");
+
+            partKey.append(column);
+        }
+
+        StringBuilder clusterKey = new StringBuilder();
+
+        cols = getClusterKeyColumns();
+        if (cols != null) {
+            for (String column : cols) {
+                if (clusterKey.length() != 0)
+                    clusterKey.append(", ");
+
+                clusterKey.append(column);
+            }
+        }
+
+        return clusterKey.length() == 0 ?
+            "  primary key ((" + partKey.toString() + "))" :
+            "  primary key ((" + partKey.toString() + "), " + clusterKey.toString() + ")";
+    }
+
+    /**
+     * Returns Cassandra DDL for cluster key.
+     *
+     * @return Cluster key DDL.
+     */
+    public String getClusteringDDL() {
+        StringBuilder builder = new StringBuilder();
+
+        for (PojoField field : clusterKeyFields) {
+            PojoKeyField.SortOrder sortOrder = ((PojoKeyField)field).getSortOrder();
+
+            if (sortOrder == null)
+                continue;
+
+            if (builder.length() != 0)
+                builder.append(", ");
+
+            boolean asc = PojoKeyField.SortOrder.ASC.equals(sortOrder);
+
+            builder.append(field.getColumn()).append(" ").append(asc ? "asc" : "desc");
+        }
+
+        return builder.length() == 0 ? null : "clustering order by (" + builder.toString() + ")";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String defaultColumnName() {
+        return "key";
+    }
+
+    /**
+     * Returns partition key columns of Cassandra table.
+     *
+     * @return List of column names.
+     */
+    private List<String> getPartitionKeyColumns() {
+        List<String> cols = new LinkedList<>();
+
+        if (PersistenceStrategy.BLOB.equals(getStrategy()) || PersistenceStrategy.PRIMITIVE.equals(getStrategy())) {
+            cols.add(getColumn());
+            return cols;
+        }
+
+        if (partKeyFields != null) {
+            for (PojoField field : partKeyFields)
+                cols.add(field.getColumn());
+        }
+
+        return cols;
+    }
+
+    /**
+     * Returns cluster key columns of Cassandra table.
+     *
+     * @return List of column names.
+     */
+    private List<String> getClusterKeyColumns() {
+        List<String> cols = new LinkedList<>();
+
+        if (clusterKeyFields != null) {
+            for (PojoField field : clusterKeyFields)
+                cols.add(field.getColumn());
+        }
+
+        return cols;
+    }
+
+    /**
+     * Extracts POJO fields specified in XML element.
+     *
+     * @param el XML element describing fields.
+     * @param descriptors POJO fields descriptors.
+     * @return List of {@code This} fields.
+     */
+    private List<PojoField> detectFields(Element el, List<PropertyDescriptor> descriptors) {
+        List<PojoField> list = new LinkedList<>();
+
+        if (el == null && (descriptors == null || descriptors.isEmpty()))
+            return list;
+
+        if (el == null) {
+            for (PropertyDescriptor descriptor : descriptors)
+                list.add(new PojoKeyField(descriptor));
+
+            return list;
+        }
+
+        NodeList nodes = el.getElementsByTagName(FIELD_ELEMENT);
+
+        int cnt = nodes == null ? 0 : nodes.getLength();
+
+        if (cnt == 0) {
+            throw new IllegalArgumentException("Incorrect configuration of Cassandra key persistence settings, " +
+                "no cluster key fields specified inside '" + PARTITION_KEY_ELEMENT + "/" +
+                CLUSTER_KEY_ELEMENT + "' element");
+        }
+
+        for (int i = 0; i < cnt; i++) {
+            PojoKeyField field = new PojoKeyField((Element)nodes.item(i), getJavaClass());
+
+            PropertyDescriptor desc = findPropertyDescriptor(descriptors, field.getName());
+
+            if (desc == null) {
+                throw new IllegalArgumentException("Specified POJO field '" + field.getName() +
+                    "' doesn't exist in '" + getJavaClass().getName() + "' class");
+            }
+
+            list.add(field);
+        }
+
+        return list;
+    }
+
+    /**
+     * @return POJO field descriptors for partition key.
+     */
+    private List<PropertyDescriptor> getPartitionKeyDescriptors() {
+        List<PropertyDescriptor> primitivePropDescriptors = PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(),
+            AffinityKeyMapped.class, true);
+
+        return primitivePropDescriptors != null && !primitivePropDescriptors.isEmpty() ?
+            primitivePropDescriptors :
+            PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), true);
+    }
+
+    /**
+     * @return POJO field descriptors for cluster key.
+     */
+    private List<PropertyDescriptor> getClusterKeyDescriptors(List<PojoField> partKeyFields) {
+        List<PropertyDescriptor> primitivePropDescriptors =
+            PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), true);
+
+        if (primitivePropDescriptors == null || primitivePropDescriptors.isEmpty() ||
+            partKeyFields.size() == primitivePropDescriptors.size())
+            return null;
+
+        for (PojoField field : partKeyFields) {
+            for (int i = 0; i < primitivePropDescriptors.size(); i++) {
+                if (primitivePropDescriptors.get(i).getName().equals(field.getName())) {
+                    primitivePropDescriptors.remove(i);
+                    break;
+                }
+            }
+        }
+
+        return primitivePropDescriptors;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java
new file mode 100644
index 0000000..2c43ed4
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java
@@ -0,0 +1,478 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Serializable;
+import java.io.StringReader;
+import java.util.LinkedList;
+import java.util.List;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.cassandra.common.SystemHelper;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.springframework.core.io.Resource;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
+
+/**
+ * Stores persistence settings for Ignite cache key and value
+ */
+public class KeyValuePersistenceSettings implements Serializable {
+    /**
+     * Default Cassandra keyspace options which should be used to create new keyspace.
+     * <ul>
+     * <li> <b>SimpleStrategy</b> for replication work well for single data center Cassandra cluster.<br/>
+     *      If your Cassandra cluster deployed across multiple data centers it's better to use <b>NetworkTopologyStrategy</b>.
+     * </li>
+     * <li> Three replicas will be created for each data block. </li>
+     * <li> Setting DURABLE_WRITES to true specifies that all data should be written to commit log. </li>
+     * </ul>
+     */
+    private static final String DFLT_KEYSPACE_OPTIONS = "replication = {'class' : 'SimpleStrategy', " +
+            "'replication_factor' : 3} and durable_writes = true";
+
+    /** Xml attribute specifying Cassandra keyspace to use. */
+    private static final String KEYSPACE_ATTR = "keyspace";
+
+    /** Xml attribute specifying Cassandra table to use. */
+    private static final String TABLE_ATTR = "table";
+
+    /** Xml attribute specifying ttl (time to leave) for rows inserted in Cassandra. */
+    private static final String TTL_ATTR = "ttl";
+
+    /** Root xml element containing persistence settings specification. */
+    private static final String PERSISTENCE_NODE = "persistence";
+
+    /** Xml element specifying Cassandra keyspace options. */
+    private static final String KEYSPACE_OPTIONS_NODE = "keyspaceOptions";
+
+    /** Xml element specifying Cassandra table options. */
+    private static final String TABLE_OPTIONS_NODE = "tableOptions";
+
+    /** Xml element specifying Ignite cache key persistence settings. */
+    private static final String KEY_PERSISTENCE_NODE = "keyPersistence";
+
+    /** Xml element specifying Ignite cache value persistence settings. */
+    private static final String VALUE_PERSISTENCE_NODE = "valuePersistence";
+
+    /** TTL (time to leave) for rows inserted into Cassandra table {@link <a href="https://docs.datastax.com/en/cql/3.1/cql/cql_using/use_expire_c.html">Expiring data</a>}. */
+    private Integer ttl;
+
+    /** Cassandra keyspace (analog of tablespace in relational databases). */
+    private String keyspace;
+
+    /** Cassandra table. */
+    private String tbl;
+
+    /** Cassandra table creation options {@link <a href="https://docs.datastax.com/en/cql/3.0/cql/cql_reference/create_table_r.html">CREATE TABLE</a>}. */
+    private String tblOptions;
+
+    /** Cassandra keyspace creation options {@link <a href="https://docs.datastax.com/en/cql/3.0/cql/cql_reference/create_keyspace_r.html">CREATE KEYSPACE</a>}. */
+    private String keyspaceOptions = DFLT_KEYSPACE_OPTIONS;
+
+    /** Persistence settings for Ignite cache keys. */
+    private KeyPersistenceSettings keyPersistenceSettings;
+
+    /** Persistence settings for Ignite cache values. */
+    private ValuePersistenceSettings valPersistenceSettings;
+
+    /**
+     * Constructs Ignite cache key/value persistence settings.
+     *
+     * @param settings string containing xml with persistence settings for Ignite cache key/value
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public KeyValuePersistenceSettings(String settings) {
+        init(settings);
+    }
+
+    /**
+     * Constructs Ignite cache key/value persistence settings.
+     *
+     * @param settingsFile xml file with persistence settings for Ignite cache key/value
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public KeyValuePersistenceSettings(File settingsFile) {
+        InputStream in;
+
+        try {
+            in = new FileInputStream(settingsFile);
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to get input stream for Cassandra persistence settings file: " +
+                    settingsFile.getAbsolutePath(), e);
+        }
+
+        init(loadSettings(in));
+    }
+
+    /**
+     * Constructs Ignite cache key/value persistence settings.
+     *
+     * @param settingsRsrc resource containing xml with persistence settings for Ignite cache key/value
+     */
+    public KeyValuePersistenceSettings(Resource settingsRsrc) {
+        InputStream in;
+
+        try {
+            in = settingsRsrc.getInputStream();
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to get input stream for Cassandra persistence settings resource: " + settingsRsrc, e);
+        }
+
+        init(loadSettings(in));
+    }
+
+    /**
+     * Returns ttl to use for while inserting new rows into Cassandra table.
+     *
+     * @return ttl
+     */
+    public Integer getTTL() {
+        return ttl;
+    }
+
+    /**
+     * Returns Cassandra keyspace to use.
+     *
+     * @return keyspace.
+     */
+    public String getKeyspace() {
+        return keyspace;
+    }
+
+    /**
+     * Returns Cassandra table to use.
+     *
+     * @return table.
+     */
+    public String getTable() {
+        return tbl;
+    }
+
+    /**
+     * Returns full name of Cassandra table to use (including keyspace).
+     *
+     * @return full table name in format "keyspace.table".
+     */
+    public String getTableFullName()
+    {
+        return keyspace + "." + tbl;
+    }
+
+    /**
+     * Returns persistence settings for Ignite cache keys.
+     *
+     * @return keys persistence settings.
+     */
+    public KeyPersistenceSettings getKeyPersistenceSettings() {
+        return keyPersistenceSettings;
+    }
+
+    /**
+     * Returns persistence settings for Ignite cache values.
+     *
+     * @return values persistence settings.
+     */
+    public ValuePersistenceSettings getValuePersistenceSettings() {
+        return valPersistenceSettings;
+    }
+
+    /**
+     * Returns list of POJO fields to be mapped to Cassandra table columns.
+     *
+     * @return POJO fields list.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public List<PojoField> getFields() {
+        List<PojoField> fields = new LinkedList<>();
+
+        for (PojoField field : keyPersistenceSettings.getFields())
+            fields.add(field);
+
+        for (PojoField field : valPersistenceSettings.getFields())
+            fields.add(field);
+
+        return fields;
+    }
+
+    /**
+     * Returns list of Ignite cache key POJO fields to be mapped to Cassandra table columns.
+     *
+     * @return POJO fields list.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public List<PojoField> getKeyFields() {
+        return keyPersistenceSettings.getFields();
+    }
+
+    /**
+     * Returns list of Ignite cache value POJO fields to be mapped to Cassandra table columns.
+     *
+     * @return POJO fields list.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public List<PojoField> getValueFields() {
+        return valPersistenceSettings.getFields();
+    }
+
+    /**
+     * Returns DDL statement to create Cassandra keyspace.
+     *
+     * @return Keyspace DDL statement.
+     */
+    public String getKeyspaceDDLStatement() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("create keyspace if not exists ").append(keyspace);
+
+        if (keyspaceOptions != null) {
+            if (!keyspaceOptions.trim().toLowerCase().startsWith("with"))
+                builder.append("\nwith");
+
+            builder.append(" ").append(keyspaceOptions);
+        }
+
+        String statement = builder.toString().trim().replaceAll(" +", " ");
+
+        return statement.endsWith(";") ? statement : statement + ";";
+    }
+
+    /**
+     * Returns DDL statement to create Cassandra table.
+     *
+     * @return Table DDL statement.
+     */
+    public String getTableDDLStatement() {
+        String colsDDL = keyPersistenceSettings.getTableColumnsDDL() + ",\n" + valPersistenceSettings.getTableColumnsDDL();
+
+        String primaryKeyDDL = keyPersistenceSettings.getPrimaryKeyDDL();
+
+        String clusteringDDL = keyPersistenceSettings.getClusteringDDL();
+
+        String optionsDDL = tblOptions != null && !tblOptions.trim().isEmpty() ? tblOptions.trim() : "";
+
+        if (clusteringDDL != null && !clusteringDDL.isEmpty())
+            optionsDDL = optionsDDL.isEmpty() ? clusteringDDL : optionsDDL + " and " + clusteringDDL;
+
+        if (!optionsDDL.trim().isEmpty())
+            optionsDDL = optionsDDL.trim().toLowerCase().startsWith("with") ? optionsDDL.trim() : "with " + optionsDDL.trim();
+
+        StringBuilder builder = new StringBuilder();
+
+        builder.append("create table if not exists ").append(keyspace).append(".").append(tbl);
+        builder.append("\n(\n").append(colsDDL).append(",\n").append(primaryKeyDDL).append("\n)");
+
+        if (!optionsDDL.isEmpty())
+            builder.append(" \n").append(optionsDDL);
+
+        String tblDDL = builder.toString().trim().replaceAll(" +", " ");
+
+        return tblDDL.endsWith(";") ? tblDDL : tblDDL + ";";
+    }
+
+    /**
+     * Returns DDL statements to create Cassandra table secondary indexes.
+     *
+     * @return DDL statements to create secondary indexes.
+     */
+    public List<String> getIndexDDLStatements() {
+        List<String> idxDDLs = new LinkedList<>();
+
+        List<PojoField> fields = valPersistenceSettings.getFields();
+
+        for (PojoField field : fields) {
+            if (((PojoValueField)field).isIndexed())
+                idxDDLs.add(((PojoValueField)field).getIndexDDL(keyspace, tbl));
+        }
+
+        return idxDDLs;
+    }
+
+    /**
+     * Loads Ignite cache persistence settings from resource.
+     *
+     * @param in Input stream.
+     * @return String containing xml with Ignite cache persistence settings.
+     */
+    private String loadSettings(InputStream in) {
+        StringBuilder settings = new StringBuilder();
+        BufferedReader reader = null;
+
+        try {
+            reader = new BufferedReader(new InputStreamReader(in));
+
+            String line = reader.readLine();
+
+            while (line != null) {
+                if (settings.length() != 0)
+                    settings.append(SystemHelper.LINE_SEPARATOR);
+
+                settings.append(line);
+
+                line = reader.readLine();
+            }
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to read input stream for Cassandra persistence settings", e);
+        }
+        finally {
+            U.closeQuiet(reader);
+            U.closeQuiet(in);
+        }
+
+        return settings.toString();
+    }
+
+    /**
+     * @param elem Element with data.
+     * @param attr Attribute name.
+     * @return Numeric value for specified attribute.
+     */
+    private int extractIntAttribute(Element elem, String attr) {
+        String val = elem.getAttribute(attr).trim();
+
+        try {
+            return Integer.parseInt(val);
+        }
+        catch (NumberFormatException e) {
+            throw new IllegalArgumentException("Incorrect value '" + val + "' specified for '" + attr + "' attribute");
+        }
+    }
+
+    /**
+     * Initializes persistence settings from XML string.
+     *
+     * @param settings XML string containing Ignite cache persistence settings configuration.
+     */
+    @SuppressWarnings("IfCanBeSwitch")
+    private void init(String settings) {
+        Document doc;
+
+        try {
+            DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+            DocumentBuilder builder = factory.newDocumentBuilder();
+            doc = builder.parse(new InputSource(new StringReader(settings)));
+        }
+        catch (Throwable e) {
+            throw new IllegalArgumentException("Failed to parse persistence settings:" +
+                SystemHelper.LINE_SEPARATOR + settings, e);
+        }
+
+        Element root = doc.getDocumentElement();
+
+        if (!PERSISTENCE_NODE.equals(root.getNodeName())) {
+            throw new IllegalArgumentException("Incorrect persistence settings specified. " +
+                "Root XML element should be 'persistence'");
+        }
+
+        if (!root.hasAttribute(KEYSPACE_ATTR)) {
+            throw new IllegalArgumentException("Incorrect persistence settings '" + KEYSPACE_ATTR +
+                "' attribute should be specified");
+        }
+
+        if (!root.hasAttribute(TABLE_ATTR)) {
+            throw new IllegalArgumentException("Incorrect persistence settings '" + TABLE_ATTR +
+                "' attribute should be specified");
+        }
+
+        keyspace = root.getAttribute(KEYSPACE_ATTR).trim();
+        tbl = root.getAttribute(TABLE_ATTR).trim();
+
+        if (root.hasAttribute(TTL_ATTR))
+            ttl = extractIntAttribute(root, TTL_ATTR);
+
+        if (!root.hasChildNodes()) {
+            throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+                "there are no key and value persistence settings specified");
+        }
+
+        NodeList children = root.getChildNodes();
+        int cnt = children.getLength();
+
+        for (int i = 0; i < cnt; i++) {
+            Node node = children.item(i);
+
+            if (node.getNodeType() != Node.ELEMENT_NODE)
+                continue;
+
+            Element el = (Element)node;
+            String nodeName = el.getNodeName();
+
+            if (nodeName.equals(TABLE_OPTIONS_NODE)) {
+                tblOptions = el.getTextContent();
+                tblOptions = tblOptions.replace("\n", " ").replace("\r", "").replace("\t", " ");
+            }
+            else if (nodeName.equals(KEYSPACE_OPTIONS_NODE)) {
+                keyspaceOptions = el.getTextContent();
+                keyspaceOptions = keyspaceOptions.replace("\n", " ").replace("\r", "").replace("\t", " ");
+            }
+            else if (nodeName.equals(KEY_PERSISTENCE_NODE))
+                keyPersistenceSettings = new KeyPersistenceSettings(el);
+            else if (nodeName.equals(VALUE_PERSISTENCE_NODE))
+                valPersistenceSettings = new ValuePersistenceSettings(el);
+        }
+
+        if (keyPersistenceSettings == null) {
+            throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+                "there are no key persistence settings specified");
+        }
+
+        if (valPersistenceSettings == null) {
+            throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+                "there are no value persistence settings specified");
+        }
+
+        List<PojoField> keyFields = keyPersistenceSettings.getFields();
+        List<PojoField> valFields = valPersistenceSettings.getFields();
+
+        if (PersistenceStrategy.POJO.equals(keyPersistenceSettings.getStrategy()) &&
+            (keyFields == null || keyFields.isEmpty())) {
+            throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+                "there are no key fields found");
+        }
+
+        if (PersistenceStrategy.POJO.equals(valPersistenceSettings.getStrategy()) &&
+            (valFields == null || valFields.isEmpty())) {
+            throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+                "there are no value fields found");
+        }
+
+        if (keyFields == null || keyFields.isEmpty() || valFields == null || valFields.isEmpty())
+            return;
+
+        for (PojoField keyField : keyFields) {
+            for (PojoField valField : valFields) {
+                if (keyField.getColumn().equals(valField.getColumn())) {
+                    throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+                        "key column '" + keyField.getColumn() + "' also specified as a value column");
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java
new file mode 100644
index 0000000..e734ca3
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java
@@ -0,0 +1,421 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+
+/**
+ * Intermediate layer between persistent store (Cassandra) and Ignite cache key/value classes.
+ * Handles  all the mappings to/from Java classes into Cassandra and responsible for all the details
+ * of how Java objects should be written/loaded to/from Cassandra.
+ */
+public class PersistenceController {
+    /** Ignite cache key/value persistence settings. */
+    private KeyValuePersistenceSettings persistenceSettings;
+
+    /** CQL statement to insert row into Cassandra table. */
+    private String writeStatement;
+
+    /** CQL statement to delete row from Cassandra table. */
+    private String delStatement;
+
+    /** CQL statement to select value fields from Cassandra table. */
+    private String loadStatement;
+
+    /** CQL statement to select key/value fields from Cassandra table. */
+    private String loadStatementWithKeyFields;
+
+    /**
+     * Constructs persistence controller from Ignite cache persistence settings.
+     *
+     * @param settings persistence settings.
+     */
+    public PersistenceController(KeyValuePersistenceSettings settings) {
+        if (settings == null)
+            throw new IllegalArgumentException("Persistent settings can't be null");
+
+        this.persistenceSettings = settings;
+    }
+
+    /**
+     * Returns Ignite cache persistence settings.
+     *
+     * @return persistence settings.
+     */
+    public KeyValuePersistenceSettings getPersistenceSettings() {
+        return persistenceSettings;
+    }
+
+    /**
+     * Returns Cassandra keyspace to use.
+     *
+     * @return keyspace.
+     */
+    public String getKeyspace() {
+        return persistenceSettings.getKeyspace();
+    }
+
+    /**
+     * Returns Cassandra table to use.
+     *
+     * @return table.
+     */
+    public String getTable() {
+        return persistenceSettings.getTable();
+    }
+
+    /**
+     * Returns CQL statement to insert row into Cassandra table.
+     *
+     * @return CQL statement.
+     */
+    public String getWriteStatement() {
+        if (writeStatement != null)
+            return writeStatement;
+
+        List<String> cols = getKeyValueColumns();
+
+        StringBuilder colsList = new StringBuilder();
+        StringBuilder questionsList = new StringBuilder();
+
+        for (String column : cols) {
+            if (colsList.length() != 0) {
+                colsList.append(", ");
+                questionsList.append(",");
+            }
+
+            colsList.append(column);
+            questionsList.append("?");
+        }
+
+        writeStatement = "insert into " + persistenceSettings.getKeyspace() + "." + persistenceSettings.getTable() + " (" +
+            colsList.toString() + ") values (" + questionsList.toString() + ")";
+
+        if (persistenceSettings.getTTL() != null)
+            writeStatement += " using ttl " + persistenceSettings.getTTL();
+
+        writeStatement += ";";
+
+        return writeStatement;
+    }
+
+    /**
+     * Returns CQL statement to delete row from Cassandra table.
+     *
+     * @return CQL statement.
+     */
+    public String getDeleteStatement() {
+        if (delStatement != null)
+            return delStatement;
+
+        List<String> cols = getKeyColumns();
+
+        StringBuilder statement = new StringBuilder();
+
+        for (String column : cols) {
+            if (statement.length() != 0)
+                statement.append(" and ");
+
+            statement.append(column).append("=?");
+        }
+
+        statement.append(";");
+
+        delStatement = "delete from " +
+            persistenceSettings.getKeyspace() + "." +
+            persistenceSettings.getTable() + " where " +
+            statement.toString();
+
+        return delStatement;
+    }
+
+    /**
+     * Returns CQL statement to select key/value fields from Cassandra table.
+     *
+     * @param includeKeyFields whether to include/exclude key fields from the returned row.
+     *
+     * @return CQL statement.
+     */
+    public String getLoadStatement(boolean includeKeyFields) {
+        if (loadStatement != null && loadStatementWithKeyFields != null)
+            return includeKeyFields ? loadStatementWithKeyFields : loadStatement;
+
+        List<String> valCols = getValueColumns();
+
+        List<String> keyCols = getKeyColumns();
+
+        StringBuilder hdrWithKeyFields = new StringBuilder("select ");
+
+        for (int i = 0; i < keyCols.size(); i++) {
+            if (i > 0)
+                hdrWithKeyFields.append(", ");
+
+            hdrWithKeyFields.append(keyCols.get(i));
+        }
+
+        StringBuilder hdr = new StringBuilder("select ");
+
+        for (int i = 0; i < valCols.size(); i++) {
+            if (i > 0)
+                hdr.append(", ");
+
+            hdrWithKeyFields.append(",");
+
+            hdr.append(valCols.get(i));
+            hdrWithKeyFields.append(valCols.get(i));
+        }
+
+        StringBuilder statement = new StringBuilder();
+
+        statement.append(" from ");
+        statement.append(persistenceSettings.getKeyspace());
+        statement.append(".").append(persistenceSettings.getTable());
+        statement.append(" where ");
+
+        for (int i = 0; i < keyCols.size(); i++) {
+            if (i > 0)
+                statement.append(" and ");
+
+            statement.append(keyCols.get(i)).append("=?");
+        }
+
+        statement.append(";");
+
+        loadStatement = hdr.toString() + statement.toString();
+        loadStatementWithKeyFields = hdrWithKeyFields.toString() + statement.toString();
+
+        return includeKeyFields ? loadStatementWithKeyFields : loadStatement;
+    }
+
+    /**
+     * Binds Ignite cache key object to {@link com.datastax.driver.core.PreparedStatement}.
+     *
+     * @param statement statement to which key object should be bind.
+     * @param key key object.
+     *
+     * @return statement with bounded key.
+     */
+    public BoundStatement bindKey(PreparedStatement statement, Object key) {
+        KeyPersistenceSettings settings = persistenceSettings.getKeyPersistenceSettings();
+
+        Object[] values = getBindingValues(settings.getStrategy(),
+            settings.getSerializer(), settings.getFields(), key);
+
+        return statement.bind(values);
+    }
+
+    /**
+     * Binds Ignite cache key and value object to {@link com.datastax.driver.core.PreparedStatement}.
+     *
+     * @param statement statement to which key and value object should be bind.
+     * @param key key object.
+     * @param val value object.
+     *
+     * @return statement with bounded key and value.
+     */
+    public BoundStatement bindKeyValue(PreparedStatement statement, Object key, Object val) {
+        KeyPersistenceSettings keySettings = persistenceSettings.getKeyPersistenceSettings();
+        Object[] keyValues = getBindingValues(keySettings.getStrategy(),
+            keySettings.getSerializer(), keySettings.getFields(), key);
+
+        ValuePersistenceSettings valSettings = persistenceSettings.getValuePersistenceSettings();
+        Object[] valValues = getBindingValues(valSettings.getStrategy(),
+            valSettings.getSerializer(), valSettings.getFields(), val);
+
+        Object[] values = new Object[keyValues.length + valValues.length];
+
+        int i = 0;
+
+        for (Object keyVal : keyValues) {
+            values[i] = keyVal;
+            i++;
+        }
+
+        for (Object valVal : valValues) {
+            values[i] = valVal;
+            i++;
+        }
+
+        return statement.bind(values);
+    }
+
+    /**
+     * Builds Ignite cache key object from returned Cassandra table row.
+     *
+     * @param row Cassandra table row.
+     *
+     * @return key object.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    public Object buildKeyObject(Row row) {
+        return buildObject(row, persistenceSettings.getKeyPersistenceSettings());
+    }
+
+    /**
+     * Builds Ignite cache value object from Cassandra table row .
+     *
+     * @param row Cassandra table row.
+     *
+     * @return value object.
+     */
+    public Object buildValueObject(Row row) {
+        return buildObject(row, persistenceSettings.getValuePersistenceSettings());
+    }
+
+    /**
+     * Builds object from Cassandra table row.
+     *
+     * @param row Cassandra table row.
+     * @param settings persistence settings to use.
+     *
+     * @return object.
+     */
+    private Object buildObject(Row row, PersistenceSettings settings) {
+        if (row == null)
+            return null;
+
+        PersistenceStrategy stgy = settings.getStrategy();
+
+        Class clazz = settings.getJavaClass();
+
+        String col = settings.getColumn();
+
+        List<PojoField> fields = settings.getFields();
+
+        if (PersistenceStrategy.PRIMITIVE.equals(stgy))
+            return PropertyMappingHelper.getCassandraColumnValue(row, col, clazz, null);
+
+        if (PersistenceStrategy.BLOB.equals(stgy))
+            return settings.getSerializer().deserialize(row.getBytes(col));
+
+        Object obj;
+
+        try {
+            obj = clazz.newInstance();
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to instantiate object of type '" + clazz.getName() + "' using reflection", e);
+        }
+
+        for (PojoField field : fields)
+            field.setValueFromRow(row, obj, settings.getSerializer());
+
+        return obj;
+    }
+
+    /**
+     * Extracts field values from POJO object and converts them into Java types
+     * which could be mapped to Cassandra types.
+     *
+     * @param stgy persistence strategy to use.
+     * @param serializer serializer to use for BLOBs.
+     * @param fields fields who's values should be extracted.
+     * @param obj object instance who's field values should be extracted.
+     *
+     * @return array of object field values converted into Java object instances having Cassandra compatible types
+     */
+    private Object[] getBindingValues(PersistenceStrategy stgy, Serializer serializer, List<PojoField> fields, Object obj) {
+        if (PersistenceStrategy.PRIMITIVE.equals(stgy)) {
+            if (PropertyMappingHelper.getCassandraType(obj.getClass()) == null ||
+                obj.getClass().equals(ByteBuffer.class) || obj instanceof byte[]) {
+                throw new IllegalArgumentException("Couldn't deserialize instance of class '" +
+                    obj.getClass().getName() + "' using PRIMITIVE strategy. Please use BLOB strategy for this case.");
+            }
+
+            return new Object[] {obj};
+        }
+
+        if (PersistenceStrategy.BLOB.equals(stgy))
+            return new Object[] {serializer.serialize(obj)};
+
+        Object[] values = new Object[fields.size()];
+
+        int i = 0;
+
+        for (PojoField field : fields) {
+            Object val = field.getValueFromObject(obj, serializer);
+
+            if (val instanceof byte[])
+                val = ByteBuffer.wrap((byte[]) val);
+
+            values[i] = val;
+
+            i++;
+        }
+
+        return values;
+    }
+
+    /**
+     * Returns list of Cassandra table columns mapped to Ignite cache key and value fields
+     *
+     * @return list of column names
+     */
+    private List<String> getKeyValueColumns() {
+        List<String> cols = getKeyColumns();
+
+        cols.addAll(getValueColumns());
+
+        return cols;
+    }
+
+    /**
+     * Returns list of Cassandra table columns mapped to Ignite cache key fields
+     *
+     * @return list of column names
+     */
+    private List<String> getKeyColumns() {
+        return getColumns(persistenceSettings.getKeyPersistenceSettings());
+    }
+
+    /**
+     * Returns list of Cassandra table columns mapped to Ignite cache value fields
+     *
+     * @return list of column names
+     */
+    private List<String> getValueColumns() {
+        return getColumns(persistenceSettings.getValuePersistenceSettings());
+    }
+
+    /**
+     * Returns list of Cassandra table columns based on persistence strategy to use
+     *
+     * @return list of column names
+     */
+    private List<String> getColumns(PersistenceSettings settings) {
+        List<String> cols = new LinkedList<>();
+
+        if (!PersistenceStrategy.POJO.equals(settings.getStrategy())) {
+            cols.add(settings.getColumn());
+            return cols;
+        }
+
+        for (PojoField field : settings.getFields())
+            cols.add(field.getColumn());
+
+        return cols;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java
new file mode 100644
index 0000000..20d790a
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java
@@ -0,0 +1,335 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import com.datastax.driver.core.DataType;
+import java.beans.PropertyDescriptor;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+import org.w3c.dom.Element;
+
+/**
+ * Stores persistence settings, which describes how particular key/value
+ * from Ignite cache should be stored in Cassandra.
+ */
+public abstract class PersistenceSettings implements Serializable {
+    /** Xml attribute specifying persistence strategy. */
+    private static final String STRATEGY_ATTR = "strategy";
+
+    /** Xml attribute specifying Cassandra column name. */
+    private static final String COLUMN_ATTR = "column";
+
+    /** Xml attribute specifying BLOB serializer to use. */
+    private static final String SERIALIZER_ATTR = "serializer";
+
+    /** Xml attribute specifying java class of the object to be persisted. */
+    private static final String CLASS_ATTR = "class";
+
+    /** Persistence strategy to use. */
+    private PersistenceStrategy stgy;
+
+    /** Java class of the object to be persisted. */
+    private Class javaCls;
+
+    /** Cassandra table column name where object should be persisted in
+     *  case of using BLOB or PRIMITIVE persistence strategy. */
+    private String col;
+
+    /** Serializer for BLOBs. */
+    private Serializer serializer = new JavaSerializer();
+
+    /**
+     * Extracts property descriptor from the descriptors list by its name.
+     *
+     * @param descriptors descriptors list.
+     * @param propName property name.
+     *
+     * @return property descriptor.
+     */
+    public static PropertyDescriptor findPropertyDescriptor(List<PropertyDescriptor> descriptors, String propName) {
+        if (descriptors == null || descriptors.isEmpty() || propName == null || propName.trim().isEmpty())
+            return null;
+
+        for (PropertyDescriptor descriptor : descriptors) {
+            if (descriptor.getName().equals(propName))
+                return descriptor;
+        }
+
+        return null;
+    }
+
+    /**
+     * Constructs persistence settings from corresponding XML element.
+     *
+     * @param el xml element containing persistence settings configuration.
+     */
+    @SuppressWarnings("unchecked")
+    public PersistenceSettings(Element el) {
+        if (el == null)
+            throw new IllegalArgumentException("DOM element representing key/value persistence object can't be null");
+
+        if (!el.hasAttribute(STRATEGY_ATTR)) {
+            throw new IllegalArgumentException("DOM element representing key/value persistence object should have '" +
+                STRATEGY_ATTR + "' attribute");
+        }
+
+        try {
+            stgy = PersistenceStrategy.valueOf(el.getAttribute(STRATEGY_ATTR).trim().toUpperCase());
+        }
+        catch (IllegalArgumentException e) {
+            throw new IllegalArgumentException("Incorrect persistence strategy specified: " + el.getAttribute(STRATEGY_ATTR));
+        }
+
+        if (!el.hasAttribute(CLASS_ATTR) && !PersistenceStrategy.BLOB.equals(stgy)) {
+            throw new IllegalArgumentException("DOM element representing key/value persistence object should have '" +
+                CLASS_ATTR + "' attribute or have BLOB persistence strategy");
+        }
+
+        try {
+            javaCls = el.hasAttribute(CLASS_ATTR) ? getClassInstance(el.getAttribute(CLASS_ATTR).trim()) : null;
+        }
+        catch (Throwable e) {
+            throw new IllegalArgumentException("Incorrect java class specified '" + el.getAttribute(CLASS_ATTR) + "' " +
+                "for Cassandra persistence", e);
+        }
+
+        if (!PersistenceStrategy.BLOB.equals(stgy) &&
+            (ByteBuffer.class.equals(javaCls) || byte[].class.equals(javaCls))) {
+            throw new IllegalArgumentException("Java class '" + el.getAttribute(CLASS_ATTR) + "' " +
+                "specified could only be persisted using BLOB persistence strategy");
+        }
+
+        if (PersistenceStrategy.PRIMITIVE.equals(stgy) &&
+            PropertyMappingHelper.getCassandraType(javaCls) == null) {
+            throw new IllegalArgumentException("Current implementation doesn't support persisting '" +
+                javaCls.getName() + "' object using PRIMITIVE strategy");
+        }
+
+        if (PersistenceStrategy.POJO.equals(stgy)) {
+            if (javaCls == null)
+                throw new IllegalStateException("Object java class should be specified for POJO persistence strategy");
+
+            try {
+                javaCls.getConstructor();
+            }
+            catch (Throwable e) {
+                throw new IllegalArgumentException("Java class '" + javaCls.getName() + "' couldn't be used as POJO " +
+                    "cause it doesn't have no arguments constructor", e);
+            }
+        }
+
+        if (el.hasAttribute(COLUMN_ATTR)) {
+            if (!PersistenceStrategy.BLOB.equals(stgy) && !PersistenceStrategy.PRIMITIVE.equals(stgy)) {
+                throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " +
+                    "'" + COLUMN_ATTR + "' attribute is only applicable for PRIMITIVE or BLOB strategy");
+            }
+
+            col = el.getAttribute(COLUMN_ATTR).trim();
+        }
+
+        if (el.hasAttribute(SERIALIZER_ATTR)) {
+            if (!PersistenceStrategy.BLOB.equals(stgy) && !PersistenceStrategy.POJO.equals(stgy)) {
+                throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " +
+                    "'" + SERIALIZER_ATTR + "' attribute is only applicable for BLOB and POJO strategies");
+            }
+
+            Object obj = newObjectInstance(el.getAttribute(SERIALIZER_ATTR).trim());
+
+            if (!(obj instanceof Serializer)) {
+                throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " +
+                    "serializer class '" + el.getAttribute(SERIALIZER_ATTR) + "' doesn't implement '" +
+                    Serializer.class.getName() + "' interface");
+            }
+
+            serializer = (Serializer)obj;
+        }
+
+        if ((PersistenceStrategy.BLOB.equals(stgy) || PersistenceStrategy.PRIMITIVE.equals(stgy)) && col == null)
+            col = defaultColumnName();
+    }
+
+    /**
+     * Returns java class of the object to be persisted.
+     *
+     * @return java class.
+     */
+    public Class getJavaClass() {
+        return javaCls;
+    }
+
+    /**
+     * Returns persistence strategy to use.
+     *
+     * @return persistence strategy.
+     */
+    public PersistenceStrategy getStrategy() {
+        return stgy;
+    }
+
+    /**
+     * Returns Cassandra table column name where object should be persisted in
+     * case of using BLOB or PRIMITIVE persistence strategy.
+     *
+     * @return column name.
+     */
+    public String getColumn() {
+        return col;
+    }
+
+    /**
+     * Returns serializer to be used for BLOBs.
+     *
+     * @return serializer.
+     */
+    public Serializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Returns list of POJO fields to be persisted.
+     *
+     * @return list of fields.
+     */
+    public abstract List<PojoField> getFields();
+
+    /**
+     * Returns Cassandra table columns DDL, corresponding to POJO fields which should be persisted.
+     *
+     * @return DDL statement for Cassandra table fields
+     */
+    public String getTableColumnsDDL() {
+        if (PersistenceStrategy.BLOB.equals(stgy))
+            return "  " + col + " " + DataType.Name.BLOB.toString();
+
+        if (PersistenceStrategy.PRIMITIVE.equals(stgy))
+            return "  " + col + " " + PropertyMappingHelper.getCassandraType(javaCls);
+
+        StringBuilder builder = new StringBuilder();
+
+        for (PojoField field : getFields()) {
+            if (builder.length() > 0)
+                builder.append(",\n");
+
+            builder.append("  ").append(field.getColumnDDL());
+        }
+
+        if (builder.length() == 0) {
+            throw new IllegalStateException("There are no POJO fields found for '" + javaCls.toString()
+                + "' class to be presented as a Cassandra primary key");
+        }
+
+        return builder.toString();
+    }
+
+    /**
+     * Returns default name for Cassandra column (if it's not specified explicitly).
+     *
+     * @return column name
+     */
+    protected abstract String defaultColumnName();
+
+    /**
+     * Checks if there are POJO filed with the same name or same Cassandra column specified in persistence settings
+     *
+     * @param fields list of fields to be persisted into Cassandra
+     */
+    protected void checkDuplicates(List<PojoField> fields) {
+        if (fields == null || fields.isEmpty())
+            return;
+
+        for (PojoField field1 : fields) {
+            boolean sameNames = false;
+            boolean sameCols = false;
+
+            for (PojoField field2 : fields) {
+                if (field1.getName().equals(field2.getName())) {
+                    if (sameNames) {
+                        throw new IllegalArgumentException("Incorrect Cassandra key persistence settings, " +
+                            "two POJO fields with the same name '" + field1.getName() + "' specified");
+                    }
+
+                    sameNames = true;
+                }
+
+                if (field1.getColumn().equals(field2.getColumn())) {
+                    if (sameCols) {
+                        throw new IllegalArgumentException("Incorrect Cassandra persistence settings, " +
+                            "two POJO fields with the same column '" + field1.getColumn() + "' specified");
+                    }
+
+                    sameCols = true;
+                }
+            }
+        }
+    }
+
+    /**
+     * Instantiates Class object for particular class
+     *
+     * @param clazz class name
+     * @return Class object
+     */
+    private Class getClassInstance(String clazz) {
+        try {
+            return Class.forName(clazz);
+        }
+        catch (ClassNotFoundException ignored) {
+        }
+
+        try {
+            return Class.forName(clazz, true, Thread.currentThread().getContextClassLoader());
+        }
+        catch (ClassNotFoundException ignored) {
+        }
+
+        try {
+            return Class.forName(clazz, true, PersistenceSettings.class.getClassLoader());
+        }
+        catch (ClassNotFoundException ignored) {
+        }
+
+        try {
+            return Class.forName(clazz, true, ClassLoader.getSystemClassLoader());
+        }
+        catch (ClassNotFoundException ignored) {
+        }
+
+        throw new IgniteException("Failed to load class '" + clazz + "' using reflection");
+    }
+
+    /**
+     * Creates new object instance of particular class
+     *
+     * @param clazz class name
+     * @return object
+     */
+    private Object newObjectInstance(String clazz) {
+        try {
+            return getClassInstance(clazz).newInstance();
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to instantiate class '" + clazz + "' using default constructor", e);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java
new file mode 100644
index 0000000..4b1e2d8
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+/**
+ * Describes persistence strategy to be used to persist object data into Cassandra.
+ */
+public enum PersistenceStrategy {
+    /**
+     * Stores object value as is, by mapping its value to Cassandra table column with corresponding type.
+     * <p>
+     * Could be used for primitive java type (like Integer, String, Long and etc) which could be directly mapped
+     * to appropriate Cassandra types.
+     */
+    PRIMITIVE,
+
+    /**
+     * Stores object value as BLOB, by mapping its value to Cassandra table column with blob type.
+     * Could be used for any java type. Conversion of java object to BLOB is handled by specified serializer.
+     * <p>
+     * Available serializer implementations:
+     * <ul>
+     *     <li>
+     *         org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer - uses standard Java
+     *         serialization framework.
+     *     </li>
+     *     <li>
+     *        org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer - uses Kryo serialization
+     *        framework.
+     *     </li>
+     * </ul>
+     */
+    BLOB,
+
+    /**
+     * Stores each field of an object as a column having corresponding type in Cassandra table.
+     * Provides ability to utilize Cassandra secondary indexes for object fields.
+     * <p>
+     * Could be used for objects which follow JavaBeans convention and having empty public constructor.
+     * Object fields should be:
+     * <ul>
+     *     <li>Primitive java types like int, long, String and etc.</li>
+     *     <li>Collections of primitive java types like List<Integer>, Map<Integer, String>, Set<Long></li>
+     * </ul>
+     */
+    POJO
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83c26a91/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java
new file mode 100644
index 0000000..af569fd
--- /dev/null
+++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
+
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.Row;
+import java.beans.PropertyDescriptor;
+import java.io.Serializable;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+import org.w3c.dom.Element;
+
+/**
+ * Descriptor for particular field in a POJO object, specifying how this field
+ * should be written to or loaded from Cassandra.
+ */
+public abstract class PojoField implements Serializable {
+    /** Name attribute of XML element describing Pojo field. */
+    private static final String NAME_ATTR = "name";
+
+    /** Column attribute of XML element describing Pojo field. */
+    private static final String COLUMN_ATTR = "column";
+
+    /** Field name. */
+    private String name;
+
+    /** Java class to which the field belongs. */
+    private Class javaCls;
+
+    /** Field column name in Cassandra table. */
+    private String col;
+
+    /** Field column DDL.  */
+    private String colDDL;
+
+    /** Field property descriptor. */
+    private transient PropertyDescriptor desc;
+
+    /**
+     * Creates instance of {@link PojoField} based on it's description in XML element.
+     *
+     * @param el XML element describing Pojo field
+     * @param pojoCls Pojo java class.
+     */
+    public PojoField(Element el, Class<?> pojoCls) {
+        if (el == null)
+            throw new IllegalArgumentException("DOM element representing POJO field object can't be null");
+
+        if (!el.hasAttribute(NAME_ATTR)) {
+            throw new IllegalArgumentException("DOM element representing POJO field object should have '"
+                + NAME_ATTR + "' attribute");
+        }
+
+        this.name = el.getAttribute(NAME_ATTR).trim();
+        this.col = el.hasAttribute(COLUMN_ATTR) ? el.getAttribute(COLUMN_ATTR).trim() : name.toLowerCase();
+
+        init(PropertyMappingHelper.getPojoPropertyDescriptor(pojoCls, name));
+    }
+
+    /**
+     * Creates instance of {@link PojoField}  from its property descriptor.
+     *
+     * @param desc Field property descriptor.
+     */
+    public PojoField(PropertyDescriptor desc) {
+        this.name = desc.getName();
+
+        QuerySqlField sqlField = desc.getReadMethod() != null ?
+            desc.getReadMethod().getAnnotation(QuerySqlField.class) :
+            desc.getWriteMethod() == null ?
+                null :
+                desc.getWriteMethod().getAnnotation(QuerySqlField.class);
+
+        this.col = sqlField != null && sqlField.name() != null ? sqlField.name() : name.toLowerCase();
+
+        init(desc);
+
+        if (sqlField != null)
+            init(sqlField);
+    }
+
+    /**
+     * @return field name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @return Cassandra table column name.
+     */
+    public String getColumn() {
+        return col;
+    }
+
+    /**
+     * @return Cassandra table column DDL statement.
+     */
+    public String getColumnDDL() {
+        return colDDL;
+    }
+
+    /**
+     * Gets field value as an object having Cassandra compatible type.
+     * This it could be stored directly into Cassandra without any conversions.
+     *
+     * @param obj Object instance.
+     * @param serializer {@link org.apache.ignite.cache.store.cassandra.serializer.Serializer} to use.
+     * @return Object to store in Cassandra table column.
+     */
+    public Object getValueFromObject(Object obj, Serializer serializer) {
+        try {
+            Object val = propDesc().getReadMethod().invoke(obj);
+
+            if (val == null)
+                return null;
+
+            DataType.Name cassandraType = PropertyMappingHelper.getCassandraType(val.getClass());
+
+            if (cassandraType != null)
+                return val;
+
+            if (serializer == null) {
+                throw new IllegalStateException("Can't serialize value from object '" +
+                    val.getClass().getName() + "' field '" + name + "', cause there is no BLOB serializer specified");
+            }
+
+            return serializer.serialize(val);
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to get value of the field '" + name + "' from the instance " +
+                " of '" + obj.getClass().toString() + "' class", e);
+        }
+    }
+
+    /**
+     * Sets object field value from a {@link com.datastax.driver.core.Row} returned by Cassandra CQL statement.
+     *
+     * @param row {@link com.datastax.driver.core.Row}
+     * @param obj object which field should be populated from {@link com.datastax.driver.core.Row}
+     * @param serializer {@link org.apache.ignite.cache.store.cassandra.serializer.Serializer} to use.
+     */
+    public void setValueFromRow(Row row, Object obj, Serializer serializer) {
+        Object val = PropertyMappingHelper.getCassandraColumnValue(row, col, propDesc().getPropertyType(), serializer);
+
+        try {
+            propDesc().getWriteMethod().invoke(obj, val);
+        }
+        catch (Throwable e) {
+            throw new IgniteException("Failed to set value of the field '" + name + "' of the instance " +
+                " of '" + obj.getClass().toString() + "' class", e);
+        }
+    }
+
+    /**
+     * Initializes field info from annotation.
+     *
+     * @param sqlField {@link QuerySqlField} annotation.
+     */
+    protected abstract void init(QuerySqlField sqlField);
+
+    /**
+     * Initializes field info from property descriptor.
+     *
+     * @param desc {@link PropertyDescriptor} descriptor.
+     */
+    protected void init(PropertyDescriptor desc) {
+        if (desc.getReadMethod() == null) {
+            throw new IllegalArgumentException("Field '" + desc.getName() +
+                "' of the class instance '" + desc.getPropertyType().getName() +
+                "' doesn't provide getter method");
+        }
+
+        if (desc.getWriteMethod() == null) {
+            throw new IllegalArgumentException("Field '" + desc.getName() +
+                "' of POJO object instance of the class '" + desc.getPropertyType().getName() +
+                "' doesn't provide write method");
+        }
+
+        if (!desc.getReadMethod().isAccessible())
+            desc.getReadMethod().setAccessible(true);
+
+        if (!desc.getWriteMethod().isAccessible())
+            desc.getWriteMethod().setAccessible(true);
+
+        DataType.Name cassandraType = PropertyMappingHelper.getCassandraType(desc.getPropertyType());
+        cassandraType = cassandraType == null ? DataType.Name.BLOB : cassandraType;
+
+        this.javaCls = desc.getReadMethod().getDeclaringClass();
+        this.desc = desc;
+        this.colDDL = col + " " + cassandraType.toString();
+    }
+
+    /**
+     * Returns property descriptor of the POJO field
+     *
+     * @return Property descriptor
+     */
+    private PropertyDescriptor propDesc() {
+        return desc != null ? desc : (desc = PropertyMappingHelper.getPojoPropertyDescriptor(javaCls, name));
+    }
+}


[46/50] ignite git commit: Revert "IGNITE-3335: IGFS: Improved affinity task execution for read-only operations from clients."

Posted by vo...@apache.org.
Revert "IGNITE-3335: IGFS: Improved affinity task execution for read-only operations from clients."

This reverts commit 06d0e5112c2722dc94a754bb9b5ee952471ac2cd.


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

Branch: refs/heads/ignite-3341
Commit: bf9de9dd99dfb80f3a7f6c36241b9d8f7fd3ed83
Parents: 8a14e47
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 17 17:39:15 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 17 17:39:15 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsDeleteWorker.java       |  2 +-
 .../processors/igfs/IgfsMetaManager.java        | 53 ++++----------------
 2 files changed, 10 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf9de9dd/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
index 8c87e97..310090d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java
@@ -39,7 +39,7 @@ import java.util.concurrent.locks.ReentrantLock;
  */
 public class IgfsDeleteWorker extends IgfsThread {
     /** Awake frequency, */
-    private static final long FREQUENCY = 100;
+    private static final long FREQUENCY = 1000;
 
     /** How many files/folders to delete at once (i.e in a single transaction). */
     private static final int MAX_DELETE_BATCH = 100;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf9de9dd/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 48ad674..8f1bb69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -22,7 +22,6 @@ import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.cluster.ClusterTopologyException;
@@ -53,14 +52,14 @@ import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaFile
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
-import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor;
-import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRenameProcessor;
-import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingReplaceProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileReserveSpaceProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileUnlockProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRemoveProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingReplaceProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdatePropertiesProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor;
 import org.apache.ignite.internal.util.GridLeanMap;
@@ -153,13 +152,10 @@ public class IgfsMetaManager extends IgfsManager {
     private final boolean client;
 
     /** Compute facade. */
-    private volatile IgniteCompute compute;
+    private IgniteCompute compute;
 
     /** Compute facade for client tasks. */
-    private volatile IgniteCompute metaCompute;
-
-    /** Root partition. */
-    private volatile int rootPart = -1;
+    private IgniteCompute metaCompute;
 
     /**
      * Constructor.
@@ -264,14 +260,9 @@ public class IgfsMetaManager extends IgfsManager {
 
                 return compute().affinityCall(cfg.getMetaCacheName(), affKey, task);
             }
-            else {
-                if (metaCache.configuration().getCacheMode() == CacheMode.REPLICATED)
-                    // For replicated cache we do not bother much, just route to any data node.
-                    return metaCompute().call(task);
-                else
-                    // Otherwise we must route request to one of affinity nodes.
-                    return rootCompute().call(task);
-            }
+            else
+                // Otherwise we route to any available data node.
+                return metaCompute().call(task);
         }
         catch (ClusterTopologyException e) {
             throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes." , e);
@@ -300,7 +291,7 @@ public class IgfsMetaManager extends IgfsManager {
     /**
      * Get metadata compute facade for client tasks.
      *
-     * @return Compute facade.
+     * @return Metadata compute facade.
      */
     private IgniteCompute metaCompute() {
         assert client;
@@ -323,32 +314,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Get compute facade for root entry affinity nodes.
-     *
-     * @return Compute facade.
-     */
-    private IgniteCompute rootCompute() {
-        IgniteEx ignite = igfsCtx.kernalContext().grid();
-
-        int rootPart0 = rootPart;
-
-        if (rootPart0 == -1) {
-            rootPart0 = metaCache.affinity().partition(IgfsUtils.ROOT_ID);
-
-            rootPart = rootPart0;
-        }
-
-        Collection<ClusterNode> nodes = metaCache.affinity().mapPartitionToPrimaryAndBackups(rootPart0);
-
-        if (nodes.isEmpty())
-            throw new IgfsException("Failed to execute operation because there are no IGFS metadata nodes.");
-
-        ClusterGroup cluster = ignite.cluster().forNodes(nodes);
-
-        return ignite.compute(cluster);
-    }
-
-    /**
      * Gets file ID for specified path.
      *
      * @param path Path.


[15/50] ignite git commit: IGNITE-3215 - Added IgniteRDD.withKeepBinary method

Posted by vo...@apache.org.
IGNITE-3215 - Added IgniteRDD.withKeepBinary method


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

Branch: refs/heads/ignite-3341
Commit: 0e18941be9349fdc61bccf1768530684387c95e9
Parents: e27ba14
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Tue Jun 14 18:09:10 2016 +0300
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Jun 15 12:07:35 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spark/IgniteContext.scala |  4 +--
 .../org/apache/ignite/spark/IgniteRDD.scala     | 19 ++++++++---
 .../apache/ignite/spark/JavaIgniteContext.scala |  4 +--
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  2 ++
 .../ignite/spark/impl/IgniteAbstractRDD.scala   | 15 ++++++---
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |  5 +--
 .../spark/impl/JavaIgniteAbstractRDD.scala      | 34 --------------------
 .../org/apache/ignite/spark/IgniteRDDSpec.scala | 25 ++++++++++++--
 8 files changed, 58 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
index bd61974..1084dbe 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteContext.scala
@@ -107,7 +107,7 @@ class IgniteContext[K, V](
      * @return `IgniteRDD` instance.
      */
     def fromCache(cacheName: String): IgniteRDD[K, V] = {
-        new IgniteRDD[K, V](this, cacheName, null)
+        new IgniteRDD[K, V](this, cacheName, null, false)
     }
 
     /**
@@ -118,7 +118,7 @@ class IgniteContext[K, V](
      * @return `IgniteRDD` instance.
      */
     def fromCache(cacheCfg: CacheConfiguration[K, V]) = {
-        new IgniteRDD[K, V](this, cacheCfg.getName, cacheCfg)
+        new IgniteRDD[K, V](this, cacheCfg.getName, cacheCfg, false)
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
index fa96212..cad96b9 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/IgniteRDD.scala
@@ -45,8 +45,9 @@ import scala.collection.JavaConversions._
 class IgniteRDD[K, V] (
     val ic: IgniteContext[K, V],
     val cacheName: String,
-    val cacheCfg: CacheConfiguration[K, V]
-) extends IgniteAbstractRDD[(K, V), K, V] (ic, cacheName, cacheCfg) {
+    val cacheCfg: CacheConfiguration[K, V],
+    val keepBinary: Boolean
+) extends IgniteAbstractRDD[(K, V), K, V] (ic, cacheName, cacheCfg, keepBinary) {
     /**
      * Computes iterator based on given partition.
      *
@@ -127,7 +128,8 @@ class IgniteRDD[K, V] (
 
         qry.setArgs(args.map(_.asInstanceOf[Object]):_*)
 
-        new IgniteSqlRDD[(K, V), Cache.Entry[K, V], K, V](ic, cacheName, cacheCfg, qry, entry \u21d2 (entry.getKey, entry.getValue))
+        new IgniteSqlRDD[(K, V), Cache.Entry[K, V], K, V](ic, cacheName, cacheCfg, qry,
+            entry \u21d2 (entry.getKey, entry.getValue), keepBinary)
     }
 
     /**
@@ -144,7 +146,8 @@ class IgniteRDD[K, V] (
 
         val schema = buildSchema(ensureCache().query(qry).asInstanceOf[QueryCursorEx[java.util.List[_]]].fieldsMeta())
 
-        val rowRdd = new IgniteSqlRDD[Row, java.util.List[_], K, V](ic, cacheName, cacheCfg, qry, list \u21d2 Row.fromSeq(list))
+        val rowRdd = new IgniteSqlRDD[Row, java.util.List[_], K, V](
+            ic, cacheName, cacheCfg, qry, list \u21d2 Row.fromSeq(list), keepBinary)
 
         ic.sqlContext.createDataFrame(rowRdd, schema)
     }
@@ -290,6 +293,14 @@ class IgniteRDD[K, V] (
         ensureCache().removeAll()
     }
 
+    def withKeepBinary[K1, V1](): IgniteRDD[K1, V1] = {
+        new IgniteRDD[K1, V1](
+            ic.asInstanceOf[IgniteContext[K1, V1]],
+            cacheName,
+            cacheCfg.asInstanceOf[CacheConfiguration[K1, V1]],
+            true)
+    }
+
     /**
      * Builds spark schema from query metadata.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
index 44b1cd9..25184e7 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteContext.scala
@@ -52,10 +52,10 @@ class JavaIgniteContext[K, V](
     }
 
     def fromCache(cacheName: String): JavaIgniteRDD[K, V] =
-        JavaIgniteRDD.fromIgniteRDD(new IgniteRDD[K, V](ic, cacheName, null))
+        JavaIgniteRDD.fromIgniteRDD(new IgniteRDD[K, V](ic, cacheName, null, false))
 
     def fromCache(cacheCfg: CacheConfiguration[K, V]) =
-        JavaIgniteRDD.fromIgniteRDD(new IgniteRDD[K, V](ic, cacheCfg.getName, cacheCfg))
+        JavaIgniteRDD.fromIgniteRDD(new IgniteRDD[K, V](ic, cacheCfg.getName, cacheCfg, false))
 
     def ignite(): Ignite = ic.ignite()
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
index cac0e15..1efc6ae 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/JavaIgniteRDD.scala
@@ -96,6 +96,8 @@ class JavaIgniteRDD[K, V](override val rdd: IgniteRDD[K, V])
         savePairs(jrdd, f, overwrite = false)
 
     def clear(): Unit = rdd.clear()
+
+    def withKeepBinary[K1, V1](): JavaIgniteRDD[K1, V1] = new JavaIgniteRDD[K1, V1](rdd.withKeepBinary[K1, V1]())
 }
 
 object JavaIgniteRDD {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
index 25b3b56..9d5171c 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteAbstractRDD.scala
@@ -27,13 +27,20 @@ import scala.reflect.ClassTag
 abstract class IgniteAbstractRDD[R:ClassTag, K, V] (
     ic: IgniteContext[K, V],
     cacheName: String,
-    cacheCfg: CacheConfiguration[K, V]
+    cacheCfg: CacheConfiguration[K, V],
+    keepBinary: Boolean
 ) extends RDD[R] (ic.sparkContext, deps = Nil) {
     protected def ensureCache(): IgniteCache[K, V] = {
         // Make sure to deploy the cache
-        if (cacheCfg != null)
-            ic.ignite().getOrCreateCache(cacheCfg)
+        val cache =
+            if (cacheCfg != null)
+                ic.ignite().getOrCreateCache(cacheCfg)
+            else
+                ic.ignite().getOrCreateCache(cacheName)
+
+        if (keepBinary)
+            cache.withKeepBinary()
         else
-            ic.ignite().getOrCreateCache(cacheName)
+            cache.asInstanceOf[IgniteCache[K, V]]
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
index 762a6ed..b4579aa 100644
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
+++ b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/IgniteSqlRDD.scala
@@ -29,8 +29,9 @@ class IgniteSqlRDD[R: ClassTag, T, K, V](
     cacheName: String,
     cacheCfg: CacheConfiguration[K, V],
     qry: Query[T],
-    conv: (T) \u21d2 R
-) extends IgniteAbstractRDD[R, K, V](ic, cacheName, cacheCfg) {
+    conv: (T) \u21d2 R,
+    keepBinary: Boolean
+) extends IgniteAbstractRDD[R, K, V](ic, cacheName, cacheCfg, keepBinary) {
     override def compute(split: Partition, context: TaskContext): Iterator[R] = {
         new IgniteQueryIterator[T, R](ensureCache().query(qry).iterator(), conv)
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala b/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala
deleted file mode 100644
index 13bd3e8..0000000
--- a/modules/spark/src/main/scala/org/apache/ignite/spark/impl/JavaIgniteAbstractRDD.scala
+++ /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.spark.impl
-
-import org.apache.ignite.IgniteCache
-import org.apache.ignite.spark.IgniteRDD
-import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike}
-
-abstract class JavaIgniteAbstractRDD[K, V](val rdd: IgniteRDD[K, V])
-    extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
-
-    protected def ensureCache(): IgniteCache[K, V] = {
-        // Make sure to deploy the cache
-        if (rdd.cacheCfg != null)
-            rdd.ic.ignite().getOrCreateCache(rdd.cacheCfg)
-        else
-            rdd.ic.ignite().getOrCreateCache(rdd.cacheName)
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e18941b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
----------------------------------------------------------------------
diff --git a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
index 61040d9..8a5b355 100644
--- a/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
+++ b/modules/spark/src/test/scala/org/apache/ignite/spark/IgniteRDDSpec.scala
@@ -18,7 +18,7 @@
 package org.apache.ignite.spark
 
 import org.apache.ignite.Ignition
-import org.apache.ignite.cache.query.annotations.{QueryTextField, QuerySqlField}
+import org.apache.ignite.cache.query.annotations.{QuerySqlField, QueryTextField}
 import org.apache.ignite.configuration.{CacheConfiguration, IgniteConfiguration}
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder
@@ -26,9 +26,10 @@ import org.apache.spark.SparkContext
 import org.junit.runner.RunWith
 import org.scalatest._
 import org.scalatest.junit.JUnitRunner
-import scala.collection.JavaConversions._
 
+import scala.collection.JavaConversions._
 import IgniteRDDSpec._
+import org.apache.ignite.binary.BinaryObject
 
 import scala.annotation.meta.field
 
@@ -267,6 +268,26 @@ class IgniteRDDSpec extends FunSpec with Matchers with BeforeAndAfterAll with Be
                 sc.stop()
             }
         }
+
+        it("should properly work with binary objects") {
+            val sc = new SparkContext("local[*]", "test")
+
+            try {
+                val ic = new IgniteContext[String, Entity](sc, () \u21d2 configuration("client", client = true))
+
+                val cache = ic.fromCache(PARTITIONED_CACHE_NAME)
+
+                cache.savePairs(sc.parallelize(0 until 10, 2).map(i \u21d2 (String.valueOf(i),
+                    new Entity(i, "name" + i, i * 100))))
+
+                val res = cache.withKeepBinary[String, BinaryObject]().map(t \u21d2 t._2.field[Int]("salary")).collect()
+
+                println(res)
+            }
+            finally {
+                sc.stop()
+            }
+        }
     }
 
     override protected def beforeEach() = {


[10/50] ignite git commit: IGNITE-3272 Fixed "Memory consumption in ContinuousQueryHandler".

Posted by vo...@apache.org.
IGNITE-3272 Fixed "Memory consumption in ContinuousQueryHandler".


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

Branch: refs/heads/ignite-3341
Commit: 5f44672053e548a6e2cff881ac57d064b60066c7
Parents: 98a0990
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 14 18:17:33 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 14 18:17:33 2016 +0300

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryEntry.java   |  16 +++
 .../continuous/CacheContinuousQueryHandler.java |   2 +-
 ...niteCacheContinuousQueryBackupQueueTest.java | 135 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite3.java         |   2 +
 4 files changed, 154 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f446720/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
index 63dc4cb..74f930a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEntry.java
@@ -200,6 +200,22 @@ public class CacheContinuousQueryEntry implements GridCacheDeployable, Message {
     }
 
     /**
+     * @return If entry filtered then will return light-weight <i><b>new entry</b></i> without values and key
+     * (avoid to huge memory consumption), otherwise {@code this}.
+     */
+    CacheContinuousQueryEntry forBackupQueue() {
+        if (!isFiltered())
+            return this;
+
+        CacheContinuousQueryEntry e =
+            new CacheContinuousQueryEntry(cacheId, evtType, null, null, null, keepBinary, part, updateCntr, topVer);
+
+        e.flags = flags;
+
+        return e;
+    }
+
+    /**
      * @return {@code True} if entry sent by backup node.
      */
     boolean isBackup() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f446720/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index d0a3722..fc38eba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -740,7 +740,7 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
         if (!primary && !internal && entry.updateCounter() != -1L /* Skip init query and expire entries */) {
             entry.markBackup();
 
-            backupQueue.add(entry);
+            backupQueue.add(entry.forBackupQueue());
         }
 
         return notify;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f446720/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
new file mode 100644
index 0000000..aea1954
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryBackupQueueTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query.continuous;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import javax.cache.configuration.Factory;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryEventFilter;
+import javax.cache.event.CacheEntryUpdatedListener;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+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.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteCacheContinuousQueryBackupQueueTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Keys count. */
+    private static final int KEYS_COUNT = 1024;
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TimeUnit.MINUTES.toMillis(2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBackupQueue() throws Exception {
+        startGridsMultiThreaded(GRID_COUNT);
+
+        final CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setLocalListener(lsnr);
+        qry.setRemoteFilterFactory(new FilterFactory());
+
+        try (QueryCursor<?> ignore = grid(0).cache(null).query(qry)) {
+            for (int i = 0; i < KEYS_COUNT; i++) {
+                log.info("Put key: " + i);
+
+                for (int j = 0; j < 100; j++)
+                    grid(i % GRID_COUNT).cache(null).put(i, new byte[1024 * 50]);
+            }
+
+            log.info("Finish.");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class FilterFactory implements Factory<CacheEntryEventFilter<Object, Object>> {
+        /** {@inheritDoc} */
+        @Override public CacheEntryEventFilter<Object, Object> create() {
+            return new CacheEventFilter();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class CacheEventFilter implements CacheEntryEventFilter<Object, Object>, Serializable {
+        /** {@inheritDoc} */
+        @Override public boolean evaluate(CacheEntryEvent<?, ?> evt) {
+            return false;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class CacheEventListener implements CacheEntryUpdatedListener<Object, Object> {
+        /** {@inheritDoc} */
+        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            fail();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f446720/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
index dbef1fb..5afce19 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapTieredTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapValuesTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryBackupQueueTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
@@ -117,6 +118,7 @@ public class IgniteCacheQuerySelfTestSuite3 extends TestSuite {
         suite.addTestSuite(CacheKeepBinaryIterationStoreEnabledTest.class);
         suite.addTestSuite(CacheKeepBinaryIterationSwapEnabledTest.class);
         suite.addTestSuite(CacheKeepBinaryIterationNearEnabledTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryBackupQueueTest.class);
 
         return suite;
     }


[34/50] ignite git commit: GG-11237 KeepBinary flag will not be send to another node.

Posted by vo...@apache.org.
GG-11237 KeepBinary flag will not be send to another node.


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

Branch: refs/heads/ignite-3341
Commit: 5c3efa9a6b17ec583422e1f8f228878286fa4644
Parents: 314794b
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Jun 15 23:06:38 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Jun 15 23:06:38 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  49 ++++-----
 .../cache/transactions/IgniteTxAdapter.java     |  15 +--
 .../GridCacheLazyPlainVersionedEntry.java       | 107 +++++++++++++++++++
 .../version/GridCachePlainVersionedEntry.java   |   7 +-
 4 files changed, 140 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5c3efa9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index b5359f3..18bb630 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -54,7 +54,7 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter;
-import org.apache.ignite.internal.processors.cache.version.GridCachePlainVersionedEntry;
+import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVersionedEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
@@ -2033,8 +2033,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 // Cache is conflict-enabled.
                 if (cctx.conflictNeedResolve()) {
-                    // Get new value, optionally unmarshalling and/or transforming it.
-                    Object writeObj0;
+                    GridCacheVersionedEntryEx newEntry;
+
+                    GridTuple3<Long, Long, Boolean> expiration = ttlAndExpireTime(expiryPlc,
+                        explicitTtl,
+                        explicitExpireTime);
+
+                    // Prepare old and new entries for conflict resolution.
+                    GridCacheVersionedEntryEx oldEntry = versionedEntry(keepBinary);
 
                     if (op == GridCacheOperation.TRANSFORM) {
                         transformClo = writeObj;
@@ -2049,14 +2055,10 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                         try {
                             Object computed = entryProcessor.process(entry, invokeArgs);
 
-                            if (entry.modified()) {
-                                writeObj0 = cctx.unwrapTemporary(entry.getValue());
-                                writeObj = cctx.toCacheObject(writeObj0);
-                            }
-                            else {
+                            if (entry.modified())
+                                writeObj = cctx.toCacheObject(cctx.unwrapTemporary(entry.getValue()));
+                            else
                                 writeObj = oldVal;
-                                writeObj0 = cctx.unwrapBinaryIfNeeded(oldVal, keepBinary, false);
-                            }
 
                             key0 = entry.key();
 
@@ -2067,24 +2069,17 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                             invokeRes = new IgniteBiTuple(null, e);
 
                             writeObj = oldVal;
-                            writeObj0 = cctx.unwrapBinaryIfNeeded(oldVal, keepBinary, false);
                         }
                     }
-                    else
-                        writeObj0 = cctx.unwrapBinaryIfNeeded(writeObj, keepBinary, false);
-
-                    GridTuple3<Long, Long, Boolean> expiration = ttlAndExpireTime(expiryPlc,
-                        explicitTtl,
-                        explicitExpireTime);
 
-                    // Prepare old and new entries for conflict resolution.
-                    GridCacheVersionedEntryEx oldEntry = versionedEntry(keepBinary);
-                    GridCacheVersionedEntryEx newEntry = new GridCachePlainVersionedEntry<>(
-                        oldEntry.key(),
-                        writeObj0,
+                    newEntry = new GridCacheLazyPlainVersionedEntry<>(
+                        cctx,
+                        key,
+                        (CacheObject)writeObj,
                         expiration.get1(),
                         expiration.get2(),
-                        conflictVer != null ? conflictVer : newVer);
+                        conflictVer != null ? conflictVer : newVer,
+                        keepBinary);
 
                     // Resolve conflict.
                     conflictCtx = cctx.conflictResolve(oldEntry, newEntry, verCheck);
@@ -3549,12 +3544,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         CacheObject val = isNew ? unswap(true, false) : rawGetOrUnmarshalUnlocked(false);
 
-        return new GridCachePlainVersionedEntry<>(cctx.unwrapBinaryIfNeeded(key, keepBinary, true),
-            cctx.unwrapBinaryIfNeeded(val, keepBinary, true),
+        return new GridCacheLazyPlainVersionedEntry<>(cctx,
+            key,
+            val,
             ttlExtras(),
             expireTimeExtras(),
             ver.conflictVersion(),
-            isNew);
+            isNew,
+            keepBinary);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c3efa9a/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 3bb3a17..4c76340 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
@@ -54,7 +54,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
-import org.apache.ignite.internal.processors.cache.version.GridCachePlainVersionedEntry;
+import org.apache.ignite.internal.processors.cache.version.GridCacheLazyPlainVersionedEntry;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersionedEntryEx;
@@ -1644,14 +1644,15 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         // Construct new entry info.
         GridCacheContext entryCtx = txEntry.context();
 
-        Object newVal0 = entryCtx.unwrapBinaryIfNeeded(newVal, txEntry.keepBinary(), false);
-
-        GridCacheVersionedEntryEx newEntry = new GridCachePlainVersionedEntry(
-            oldEntry.key(),
-            newVal0,
+        GridCacheVersionedEntryEx newEntry = new GridCacheLazyPlainVersionedEntry(
+            entryCtx,
+            txEntry.key(),
+            newVal,
             newTtl,
             newExpireTime,
-            newVer);
+            newVer,
+            false,
+            txEntry.keepBinary());
 
         GridCacheVersionConflictContext ctx = old.context().conflictResolve(oldEntry, newEntry, false);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c3efa9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheLazyPlainVersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheLazyPlainVersionedEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheLazyPlainVersionedEntry.java
new file mode 100644
index 0000000..50de328
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheLazyPlainVersionedEntry.java
@@ -0,0 +1,107 @@
+/*
+ * 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.version;
+
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+
+/**
+ * Lazy plain versioned entry.
+ */
+public class GridCacheLazyPlainVersionedEntry<K, V> extends GridCachePlainVersionedEntry<K, V> {
+    /** Cache context. */
+    protected GridCacheContext cctx;
+
+    /** Key cache object. */
+    private KeyCacheObject keyObj;
+
+    /** Cache object value. */
+    private CacheObject valObj;
+
+    /** Keep binary flag. */
+    private boolean keepBinary;
+
+    /**
+     * @param cctx Context.
+     * @param keyObj Key.
+     * @param valObj Value.
+     * @param ttl TTL.
+     * @param expireTime Expire time.
+     * @param ver Version.
+     * @param isStartVer Start version flag.
+     * @param keepBinary Keep binary flag.
+     */
+    public GridCacheLazyPlainVersionedEntry(GridCacheContext cctx,
+        KeyCacheObject keyObj,
+        CacheObject valObj,
+        long ttl,
+        long expireTime,
+        GridCacheVersion ver,
+        boolean isStartVer,
+        boolean keepBinary) {
+        super(null, null, ttl, expireTime, ver, isStartVer);
+
+        this.cctx = cctx;
+        this.keyObj = keyObj;
+        this.valObj = valObj;
+        this.keepBinary = keepBinary;
+    }
+
+    public GridCacheLazyPlainVersionedEntry(GridCacheContext cctx,
+        KeyCacheObject keyObj,
+        CacheObject valObj,
+        long ttl,
+        long expireTime,
+        GridCacheVersion ver,
+        boolean keepBinary) {
+        super(null, null, ttl, expireTime, ver);
+        this.cctx = cctx;
+        this.keepBinary = keepBinary;
+        this.keyObj = keyObj;
+        this.valObj = valObj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public K key() {
+        if (key == null)
+            key = (K)cctx.unwrapBinaryIfNeeded(keyObj, keepBinary);
+
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public V value() {
+        return value(keepBinary);
+    }
+
+    /**
+     * Returns the value stored in the cache when this entry was created.
+     *
+     * @param keepBinary Flag to keep binary if needed.
+     * @return the value corresponding to this entry
+     */
+    @SuppressWarnings("unchecked")
+    public V value(boolean keepBinary) {
+        if (val == null)
+            val = (V)cctx.unwrapBinaryIfNeeded(valObj, keepBinary, true);
+
+        return val;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c3efa9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCachePlainVersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCachePlainVersionedEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCachePlainVersionedEntry.java
index e1da1c4..dd682e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCachePlainVersionedEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCachePlainVersionedEntry.java
@@ -25,10 +25,10 @@ import org.jetbrains.annotations.Nullable;
  */
 public class GridCachePlainVersionedEntry<K, V> implements GridCacheVersionedEntryEx<K, V> {
     /** Key. */
-    private final K key;
+    protected K key;
 
     /** Value. */
-    private final V val;
+    protected V val;
 
     /** TTL. */
     private final long ttl;
@@ -63,9 +63,6 @@ public class GridCachePlainVersionedEntry<K, V> implements GridCacheVersionedEnt
      */
     public GridCachePlainVersionedEntry(K key, V val, long ttl, long expireTime, GridCacheVersion ver,
         boolean isStartVer) {
-        assert ver != null;
-        assert key != null;
-
         this.key = key;
         this.val = val;
         this.ttl = ttl;


[07/50] ignite git commit: IGNITE-3294: IGFS: Created separate processor optimized for entry rename.

Posted by vo...@apache.org.
IGNITE-3294: IGFS: Created separate processor optimized for entry rename.


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

Branch: refs/heads/ignite-3341
Commit: 6625e0138e4acae557972d8c3cfc49fac4ad7574
Parents: 40e6614
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Jun 14 11:36:33 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Jun 14 11:36:33 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        |   7 +-
 ...IgfsMetaDirectoryListingRenameProcessor.java | 133 +++++++++++++++++++
 2 files changed, 136 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6625e013/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index c08d6a0..39ec182 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.igfs.client.IgfsClientAbstractCalla
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable;
 import org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor;
+import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingRenameProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileCreateProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileLockProcessor;
 import org.apache.ignite.internal.processors.igfs.meta.IgfsMetaFileReserveSpaceProcessor;
@@ -1740,10 +1741,8 @@ public class IgfsMetaManager extends IgfsManager {
         IgniteUuid destId, String destName) throws IgniteCheckedException {
         validTxState(true);
 
-        if (F.eq(srcId, destId)) {
-            id2InfoPrj.invoke(srcId, new IgfsMetaDirectoryListingRemoveProcessor(srcName, entry.fileId()));
-            id2InfoPrj.invoke(destId, new IgfsMetaDirectoryListingAddProcessor(destName, entry));
-        }
+        if (F.eq(srcId, destId))
+            id2InfoPrj.invoke(srcId, new IgfsMetaDirectoryListingRenameProcessor(srcName, destName));
         else {
 
             Map<IgniteUuid, EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>> procMap = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/6625e013/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRenameProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRenameProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRenameProcessor.java
new file mode 100644
index 0000000..6460adf
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryListingRenameProcessor.java
@@ -0,0 +1,133 @@
+/*
+ * 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.igfs.meta;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo;
+import org.apache.ignite.internal.processors.igfs.IgfsListingEntry;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Processor to rename a single entry in a directory listing.
+ */
+public class IgfsMetaDirectoryListingRenameProcessor implements EntryProcessor<IgniteUuid, IgfsEntryInfo, Void>,
+    Externalizable, Binarylizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Old name. */
+    private String oldName;
+
+    /** New name. */
+    private String newName;
+
+    /**
+     * Constructor.
+     */
+    public IgfsMetaDirectoryListingRenameProcessor() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param oldName Old name.
+     * @param newName New name.
+     */
+    public IgfsMetaDirectoryListingRenameProcessor(String oldName, String newName) {
+        this.oldName = oldName;
+        this.newName = newName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void process(MutableEntry<IgniteUuid, IgfsEntryInfo> e, Object... args)
+        throws EntryProcessorException {
+        IgfsEntryInfo fileInfo = e.getValue();
+
+        assert fileInfo.isDirectory();
+
+        Map<String, IgfsListingEntry> listing = new HashMap<>(fileInfo.listing());
+
+        // Modify listing in-place.
+        IgfsListingEntry entry = listing.remove(oldName);
+
+        if (entry == null)
+            throw new IgniteException("Directory listing doesn't contain expected entry: " + oldName);
+
+        IgfsListingEntry replacedEntry = listing.put(newName, entry);
+
+        if (replacedEntry != null)
+            throw new IgniteException("Entry with new name already exists [name=" + newName +
+                ", entry=" + replacedEntry + ']');
+
+        e.setValue(fileInfo.listing(listing));
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, oldName);
+        U.writeString(out, newName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        oldName = U.readString(in);
+        newName = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        BinaryRawWriter out = writer.rawWriter();
+
+        out.writeString(oldName);
+        out.writeString(newName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader in = reader.rawReader();
+
+        oldName = in.readString();
+        newName = in.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgfsMetaDirectoryListingRenameProcessor.class, this);
+    }
+}