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/17 12:38:48 UTC

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

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>