You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sh...@apache.org on 2016/11/07 04:30:02 UTC

[15/50] [abbrv] ignite git commit: IGNITE-3609 Utilize Cassandra logged batches for transactions. - Fixes #1111.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
index 8fdcf4c..d0a787a 100644
--- a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
@@ -18,19 +18,27 @@
 package org.apache.ignite.tests;
 
 import java.util.Collection;
+import java.util.Date;
 import java.util.Map;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
 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.internal.util.typedef.internal.U;
 import org.apache.ignite.tests.pojos.Person;
 import org.apache.ignite.tests.pojos.PersonId;
+import org.apache.ignite.tests.pojos.Product;
+import org.apache.ignite.tests.pojos.ProductOrder;
 import org.apache.ignite.tests.utils.CacheStoreHelper;
 import org.apache.ignite.tests.utils.CassandraHelper;
 import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.log4j.Logger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -244,12 +252,19 @@ public class IgnitePersistentStoreTest {
 
         Map<Long, Person> personMap1 = TestsHelper.generateLongsPersonsMap();
         Map<PersonId, Person> personMap2 = TestsHelper.generatePersonIdsPersonsMap();
+        Map<Long, Product> productsMap = TestsHelper.generateProductsMap();
+        Map<Long, ProductOrder> ordersMap = TestsHelper.generateOrdersMap();
+
+        Product product = TestsHelper.generateRandomProduct(-1L);
+        ProductOrder order = TestsHelper.generateRandomOrder(-1L);
 
         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"));
             IgniteCache<PersonId, Person> personCache4 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache4"));
+            IgniteCache<Long, Product> productCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Product>("product"));
+            IgniteCache<Long, ProductOrder> orderCache = ignite.getOrCreateCache(new CacheConfiguration<Long, ProductOrder>("order"));
 
             LOGGER.info("Running single operation write tests");
 
@@ -262,6 +277,9 @@ public class IgnitePersistentStoreTest {
             personCache3.put(id, TestsHelper.generateRandomPerson(id.getPersonNumber()));
             personCache4.put(id, TestsHelper.generateRandomPerson(id.getPersonNumber()));
 
+            productCache.put(product.getId(), product);
+            orderCache.put(order.getId(), order);
+
             LOGGER.info("Single operation write tests passed");
 
             LOGGER.info("Running bulk operation write tests");
@@ -269,6 +287,8 @@ public class IgnitePersistentStoreTest {
             personCache2.putAll(personMap2);
             personCache3.putAll(personMap2);
             personCache4.putAll(personMap2);
+            productCache.putAll(productsMap);
+            orderCache.putAll(ordersMap);
             LOGGER.info("Bulk operation write tests passed");
         }
 
@@ -283,6 +303,8 @@ public class IgnitePersistentStoreTest {
             IgniteCache<PersonId, Person> personCache2 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache2"));
             IgniteCache<PersonId, Person> personCache3 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache3"));
             IgniteCache<PersonId, Person> personCache4 = ignite.getOrCreateCache(new CacheConfiguration<PersonId, Person>("cache4"));
+            IgniteCache<Long, Product> productCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Product>("product"));
+            IgniteCache<Long, ProductOrder> orderCache = ignite.getOrCreateCache(new CacheConfiguration<Long, ProductOrder>("order"));
 
             LOGGER.info("Running single operation read tests");
             Person person = personCache1.get(1L);
@@ -303,6 +325,14 @@ public class IgnitePersistentStoreTest {
             if (!person.equals(personMap2.get(id)))
                 throw new RuntimeException("Person value was incorrectly deserialized from Cassandra");
 
+            Product product1 = productCache.get(product.getId());
+            if (!product.equals(product1))
+                throw new RuntimeException("Product value was incorrectly deserialized from Cassandra");
+
+            ProductOrder order1 = orderCache.get(order.getId());
+            if (!order.equals(order1))
+                throw new RuntimeException("Order value was incorrectly deserialized from Cassandra");
+
             LOGGER.info("Single operation read tests passed");
 
             LOGGER.info("Running bulk operation read tests");
@@ -323,6 +353,14 @@ public class IgnitePersistentStoreTest {
             if (!TestsHelper.checkPersonMapsEqual(persons4, personMap2, false))
                 throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra");
 
+            Map<Long, Product> productsMap1 = productCache.getAll(productsMap.keySet());
+            if (!TestsHelper.checkProductMapsEqual(productsMap, productsMap1))
+                throw new RuntimeException("Product values batch was incorrectly deserialized from Cassandra");
+
+            Map<Long, ProductOrder> ordersMap1 = orderCache.getAll(ordersMap.keySet());
+            if (!TestsHelper.checkOrderMapsEqual(ordersMap, ordersMap1))
+                throw new RuntimeException("Order values batch was incorrectly deserialized from Cassandra");
+
             LOGGER.info("Bulk operation read tests passed");
 
             LOGGER.info("POJO strategy read tests passed");
@@ -341,12 +379,35 @@ public class IgnitePersistentStoreTest {
             personCache4.remove(id);
             personCache4.removeAll(personMap2.keySet());
 
+            productCache.remove(product.getId());
+            productCache.removeAll(productsMap.keySet());
+
+            orderCache.remove(order.getId());
+            orderCache.removeAll(ordersMap.keySet());
+
             LOGGER.info("POJO strategy delete tests passed");
         }
     }
 
     /** */
     @Test
+    public void pojoStrategyTransactionTest() {
+        CassandraHelper.dropTestKeyspaces();
+
+        Ignition.stopAll(true);
+
+        try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) {
+            pojoStrategyTransactionTest(ignite, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED);
+            pojoStrategyTransactionTest(ignite, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ);
+            pojoStrategyTransactionTest(ignite, TransactionConcurrency.OPTIMISTIC, TransactionIsolation.SERIALIZABLE);
+            pojoStrategyTransactionTest(ignite, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.READ_COMMITTED);
+            pojoStrategyTransactionTest(ignite, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ);
+            pojoStrategyTransactionTest(ignite, TransactionConcurrency.PESSIMISTIC, TransactionIsolation.SERIALIZABLE);
+        }
+    }
+
+    /** */
+    @Test
     public void loadCacheTest() {
         Ignition.stopAll(true);
 
@@ -360,6 +421,7 @@ public class IgnitePersistentStoreTest {
 
         Collection<CacheEntryImpl<PersonId, Person>> entries = TestsHelper.generatePersonIdsPersonsEntries();
 
+        //noinspection unchecked
         store.writeAll(entries);
 
         LOGGER.info("Cassandra table filled with test data");
@@ -387,4 +449,207 @@ public class IgnitePersistentStoreTest {
 
         LOGGER.info("loadCache test passed");
     }
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private void pojoStrategyTransactionTest(Ignite ignite, TransactionConcurrency concurrency,
+                                             TransactionIsolation isolation) {
+        LOGGER.info("-----------------------------------------------------------------------------------");
+        LOGGER.info("Running POJO transaction tests using " + concurrency +
+                " concurrency and " + isolation + " isolation level");
+        LOGGER.info("-----------------------------------------------------------------------------------");
+
+        CacheStore productStore = CacheStoreHelper.createCacheStore("product",
+            new ClassPathResource("org/apache/ignite/tests/persistence/pojo/product.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        CacheStore orderStore = CacheStoreHelper.createCacheStore("order",
+            new ClassPathResource("org/apache/ignite/tests/persistence/pojo/order.xml"),
+            CassandraHelper.getAdminDataSrc());
+
+        Map<Long, Product> productsMap = TestsHelper.generateProductsMap(5);
+        Map<Long, Product> productsMap1;
+        Map<Long, ProductOrder> ordersMap = TestsHelper.generateOrdersMap(5);
+        Map<Long, ProductOrder> ordersMap1;
+        Product product = TestsHelper.generateRandomProduct(-1L);
+        ProductOrder order = TestsHelper.generateRandomOrder(-1L, -1L, new Date());
+
+        IgniteTransactions txs = ignite.transactions();
+
+        IgniteCache<Long, Product> productCache = ignite.getOrCreateCache(new CacheConfiguration<Long, Product>("product"));
+        IgniteCache<Long, ProductOrder> orderCache = ignite.getOrCreateCache(new CacheConfiguration<Long, ProductOrder>("order"));
+
+        LOGGER.info("Running POJO strategy write tests");
+
+        LOGGER.info("Running single operation write tests");
+
+        Transaction tx = txs.txStart(concurrency, isolation);
+
+        try {
+            productCache.put(product.getId(), product);
+            orderCache.put(order.getId(), order);
+
+            if (productStore.load(product.getId()) != null || orderStore.load(order.getId()) != null) {
+                throw new RuntimeException("Single write operation test failed. Transaction wasn't committed yet, but " +
+                        "objects were already persisted into Cassandra");
+            }
+
+            Map<Long, Product> products = (Map<Long, Product>)productStore.loadAll(productsMap.keySet());
+            Map<Long, ProductOrder> orders = (Map<Long, ProductOrder>)orderStore.loadAll(ordersMap.keySet());
+
+            if ((products != null && !products.isEmpty()) || (orders != null && !orders.isEmpty())) {
+                throw new RuntimeException("Single write operation test failed. Transaction wasn't committed yet, but " +
+                        "objects were already persisted into Cassandra");
+            }
+
+            tx.commit();
+        }
+        finally {
+            U.closeQuiet(tx);
+        }
+
+        Product product1 = (Product)productStore.load(product.getId());
+        ProductOrder order1 = (ProductOrder)orderStore.load(order.getId());
+
+        if (product1 == null || order1 == null) {
+            throw new RuntimeException("Single write operation test failed. Transaction was committed, but " +
+                    "no objects were persisted into Cassandra");
+        }
+
+        if (!product.equals(product1) || !order.equals(order1)) {
+            throw new RuntimeException("Single write operation test failed. Transaction was committed, but " +
+                    "objects were incorrectly persisted/loaded to/from Cassandra");
+        }
+
+        LOGGER.info("Single operation write tests passed");
+
+        LOGGER.info("Running bulk operation write tests");
+
+        tx = txs.txStart(concurrency, isolation);
+
+        try {
+            productCache.putAll(productsMap);
+            orderCache.putAll(ordersMap);
+
+            productsMap1 = (Map<Long, Product>)productStore.loadAll(productsMap.keySet());
+            ordersMap1 = (Map<Long, ProductOrder>)orderStore.loadAll(ordersMap.keySet());
+
+            if ((productsMap1 != null && !productsMap1.isEmpty()) || (ordersMap1 != null && !ordersMap1.isEmpty())) {
+                throw new RuntimeException("Bulk write operation test failed. Transaction wasn't committed yet, but " +
+                        "objects were already persisted into Cassandra");
+            }
+
+            tx.commit();
+        }
+        finally {
+            U.closeQuiet(tx);
+        }
+
+        productsMap1 = (Map<Long, Product>)productStore.loadAll(productsMap.keySet());
+        ordersMap1 = (Map<Long, ProductOrder>)orderStore.loadAll(ordersMap.keySet());
+
+        if (productsMap1 == null || productsMap1.isEmpty() || ordersMap1 == null || ordersMap1.isEmpty()) {
+            throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+                    "no objects were persisted into Cassandra");
+        }
+
+        if (productsMap1.size() < productsMap.size() || ordersMap1.size() < ordersMap.size()) {
+            throw new RuntimeException("Bulk write operation test failed. There were committed less objects " +
+                    "into Cassandra than expected");
+        }
+
+        if (productsMap1.size() > productsMap.size() || ordersMap1.size() > ordersMap.size()) {
+            throw new RuntimeException("Bulk write operation test failed. There were committed more objects " +
+                    "into Cassandra than expected");
+        }
+
+        for (Map.Entry<Long, Product> entry : productsMap.entrySet()) {
+            product = productsMap1.get(entry.getKey());
+
+            if (!entry.getValue().equals(product)) {
+                throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+                        "some objects were incorrectly persisted/loaded to/from Cassandra");
+            }
+        }
+
+        for (Map.Entry<Long, ProductOrder> entry : ordersMap.entrySet()) {
+            order = ordersMap1.get(entry.getKey());
+
+            if (!entry.getValue().equals(order)) {
+                throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+                        "some objects were incorrectly persisted/loaded to/from Cassandra");
+            }
+        }
+
+        LOGGER.info("Bulk operation write tests passed");
+
+        LOGGER.info("POJO strategy write tests passed");
+
+        LOGGER.info("Running POJO strategy delete tests");
+
+        LOGGER.info("Running single delete tests");
+
+        tx = txs.txStart(concurrency, isolation);
+
+        try {
+            productCache.remove(-1L);
+            orderCache.remove(-1L);
+
+            if (productStore.load(-1L) == null || orderStore.load(-1L) == null) {
+                throw new RuntimeException("Single delete operation test failed. Transaction wasn't committed yet, but " +
+                        "objects were already deleted from Cassandra");
+            }
+
+            tx.commit();
+        }
+        finally {
+            U.closeQuiet(tx);
+        }
+
+        if (productStore.load(-1L) != null || orderStore.load(-1L) != null) {
+            throw new RuntimeException("Single delete operation test failed. Transaction was committed, but " +
+                    "objects were not deleted from Cassandra");
+        }
+
+        LOGGER.info("Single delete tests passed");
+
+        LOGGER.info("Running bulk delete tests");
+
+        tx = txs.txStart(concurrency, isolation);
+
+        try {
+            productCache.removeAll(productsMap.keySet());
+            orderCache.removeAll(ordersMap.keySet());
+
+            productsMap1 = (Map<Long, Product>)productStore.loadAll(productsMap.keySet());
+            ordersMap1 = (Map<Long, ProductOrder>)orderStore.loadAll(ordersMap.keySet());
+
+            if (productsMap1.size() != productsMap.size() || ordersMap1.size() != ordersMap.size()) {
+                throw new RuntimeException("Bulk delete operation test failed. Transaction wasn't committed yet, but " +
+                        "objects were already deleted from Cassandra");
+            }
+
+            tx.commit();
+        }
+        finally {
+            U.closeQuiet(tx);
+        }
+
+        productsMap1 = (Map<Long, Product>)productStore.loadAll(productsMap.keySet());
+        ordersMap1 = (Map<Long, ProductOrder>)orderStore.loadAll(ordersMap.keySet());
+
+        if ((productsMap1 != null && !productsMap1.isEmpty()) || (ordersMap1 != null && !ordersMap1.isEmpty())) {
+            throw new RuntimeException("Bulk delete operation test failed. Transaction was committed, but " +
+                    "objects were not deleted from Cassandra");
+        }
+
+        LOGGER.info("Bulk delete tests passed");
+
+        LOGGER.info("POJO strategy delete tests passed");
+
+        LOGGER.info("-----------------------------------------------------------------------------------");
+        LOGGER.info("Passed POJO transaction tests for " + concurrency +
+                " concurrency and " + isolation + " isolation level");
+        LOGGER.info("-----------------------------------------------------------------------------------");
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/Product.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/Product.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/Product.java
new file mode 100644
index 0000000..f8eadf4
--- /dev/null
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/Product.java
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.ignite.cache.query.annotations.QuerySqlField;
+
+/**
+ * Simple POJO to store information about product
+ */
+public class Product {
+    /** */
+    private long id;
+
+    /** */
+    private String type;
+
+    /** */
+    private String title;
+
+    /** */
+    private String description;
+
+    /** */
+    private float price;
+
+    /** */
+    public Product() {
+    }
+
+    /** */
+    public Product(long id, String type, String title, String description, float price) {
+        this.id = id;
+        this.type = type;
+        this.title = title;
+        this.description = description;
+        this.price = price;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return ((Long)id).hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        return obj instanceof Product && id == ((Product) obj).id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return id + ", " + price + ", " + type + ", " + title + ", " + description;
+    }
+
+    /** */
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    /** */
+    @QuerySqlField(index = true)
+    public long getId() {
+        return id;
+    }
+
+    /** */
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    /** */
+    @QuerySqlField
+    public String getType() {
+        return type;
+    }
+
+    /** */
+    public void setTitle(String title) {
+        this.title = title;
+    }
+
+    /** */
+    @QuerySqlField(index = true)
+    public String getTitle() {
+        return title;
+    }
+
+    /** */
+    public void setDescription(String description) {
+        this.description = description;
+    }
+
+    /** */
+    @QuerySqlField
+    public String getDescription() {
+        return description;
+    }
+
+    /** */
+    public void setPrice(float price) {
+        this.price = price;
+    }
+
+    /** */
+    @QuerySqlField
+    public float getPrice() {
+        return price;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/ProductOrder.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/ProductOrder.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/ProductOrder.java
new file mode 100644
index 0000000..4baee83
--- /dev/null
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/pojos/ProductOrder.java
@@ -0,0 +1,148 @@
+/*
+ * 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 org.apache.ignite.cache.query.annotations.QuerySqlField;
+
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * Simple POJO to store information about product order
+ */
+public class ProductOrder {
+    /** */
+    private static final DateFormat FORMAT = new SimpleDateFormat("MM/dd/yyyy/S");
+
+    /** */
+    private static final DateFormat FULL_FORMAT = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss:S");
+
+    /** */
+    private long id;
+
+    /** */
+    private long productId;
+
+    /** */
+    private Date date;
+
+    /** */
+    private int amount;
+
+    /** */
+    private float price;
+
+    /** */
+    public ProductOrder() {
+    }
+
+    /** */
+    public ProductOrder(long id, Product product, Date date, int amount) {
+        this(id, product.getId(), product.getPrice(), date, amount);
+    }
+
+    /** */
+    public ProductOrder(long id, long productId, float productPrice, Date date, int amount) {
+        this.id = id;
+        this.productId = productId;
+        this.date = date;
+        this.amount = amount;
+        this.price = productPrice * amount;
+
+        // if user ordered more than 10 items provide 5% discount
+        if (amount > 10)
+            this.price = this.price * 0.95F;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return ((Long)id).hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        return obj instanceof ProductOrder && id == ((ProductOrder) obj).id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return id + ", " + productId + ", " + FULL_FORMAT.format(date) + ", " + getDayMillisecond() + ", " + amount + ", " + price;
+    }
+
+    /** */
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    /** */
+    @QuerySqlField(index = true)
+    public long getId() {
+        return id;
+    }
+
+    /** */
+    public void setProductId(long productId) {
+        this.productId = productId;
+    }
+
+    /** */
+    @QuerySqlField(index = true)
+    public long getProductId() {
+        return productId;
+    }
+
+    /** */
+    public void setDate(Date date) {
+        this.date = date;
+    }
+
+    /** */
+    @QuerySqlField
+    public Date getDate() {
+        return date;
+    }
+
+    /** */
+    public void setAmount(int amount) {
+        this.amount = amount;
+    }
+
+    /** */
+    @QuerySqlField
+    public int getAmount() {
+        return amount;
+    }
+
+    /** */
+    public void setPrice(float price) {
+        this.price = price;
+    }
+
+    /** */
+    @QuerySqlField
+    public float getPrice() {
+        return price;
+    }
+
+    /** */
+    @QuerySqlField
+    public String getDayMillisecond() {
+        return FORMAT.format(date);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
index b5ff5ad..9bcda6e 100644
--- a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java
@@ -19,6 +19,7 @@ package org.apache.ignite.tests.utils;
 
 import java.lang.reflect.Field;
 import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreSession;
 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;
@@ -35,12 +36,24 @@ public class CacheStoreHelper {
 
     /** */
     public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource conn) {
-        return createCacheStore(cacheName, persistenceSettings, conn, LOGGER);
+        return createCacheStore(cacheName, persistenceSettings, conn, null, LOGGER);
     }
 
     /** */
     public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource conn,
-        Logger log) {
+                                              CacheStoreSession session) {
+        return createCacheStore(cacheName, persistenceSettings, conn, session, LOGGER);
+    }
+
+    /** */
+    public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource conn,
+                                              Logger log) {
+        return createCacheStore(cacheName, persistenceSettings, conn, null, log);
+    }
+
+    /** */
+    public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource conn,
+                                              CacheStoreSession session, Logger log) {
         CassandraCacheStore<Integer, Integer> cacheStore =
             new CassandraCacheStore<>(conn, new KeyValuePersistenceSettings(persistenceSettings),
                 Runtime.getRuntime().availableProcessors());
@@ -52,7 +65,7 @@ public class CacheStoreHelper {
             sesField.setAccessible(true);
             logField.setAccessible(true);
 
-            sesField.set(cacheStore, new TestCacheSession(cacheName));
+            sesField.set(cacheStore, session != null ? session : new TestCacheSession(cacheName));
             logField.set(cacheStore, new Log4JLogger(log));
         }
         catch (Throwable e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
index 1cedb7a..3cb47e9 100644
--- a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java
@@ -34,7 +34,7 @@ public class TestCacheSession implements CacheStoreSession {
     private Transaction tx;
 
     /** */
-    private Map<Object, Object> props;
+    private Map<Object, Object> props = U.newHashMap(1);
 
     /** */
     private Object attach;
@@ -45,6 +45,13 @@ public class TestCacheSession implements CacheStoreSession {
     }
 
     /** */
+    public TestCacheSession(String cacheName, Transaction tx, Map<Object, Object> props) {
+        this.cacheName = cacheName;
+        this.tx = tx;
+        this.props = props;
+    }
+
+    /** */
     @SuppressWarnings("UnusedDeclaration")
     public void newSession(@Nullable Transaction tx) {
         this.tx = tx;
@@ -78,9 +85,6 @@ public class TestCacheSession implements CacheStoreSession {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> Map<K, V> properties() {
-        if (props == null)
-            props = U.newHashMap(1);
-
         return (Map<K, V>)props;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestTransaction.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestTransaction.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestTransaction.java
new file mode 100644
index 0000000..cda6715
--- /dev/null
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestTransaction.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.tests.utils;
+
+import org.apache.ignite.lang.IgniteAsyncSupport;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.ignite.transactions.TransactionState;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.UUID;
+
+/**
+ * Dummy transaction for test purposes.
+ */
+public class TestTransaction implements Transaction {
+    /** */
+    private final IgniteUuid xid = IgniteUuid.randomUuid();
+
+    /** {@inheritDoc} */
+    @Nullable
+    @Override public IgniteUuid xid() {
+        return xid;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID nodeId() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long threadId() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long startTime() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public TransactionIsolation isolation() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public TransactionConcurrency concurrency() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean implicit() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isInvalidate() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public TransactionState state() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long timeout() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long timeout(long timeout) {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean setRollbackOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isRollbackOnly() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void commit() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteAsyncSupport withAsync() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isAsync() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <R> IgniteFuture<R> future() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rollback() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
index 2e266f6..24d64c9 100644
--- a/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
+++ b/modules/cassandra/store/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java
@@ -17,19 +17,27 @@
 
 package org.apache.ignite.tests.utils;
 
+
+import org.apache.ignite.cache.store.cassandra.common.SystemHelper;
+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.apache.ignite.tests.pojos.Product;
+import org.apache.ignite.tests.pojos.ProductOrder;
+import org.springframework.core.io.ClassPathResource;
+
 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.HashMap;
+import java.util.Set;
+import java.util.HashSet;
 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;
+import java.util.Calendar;
+import java.util.Date;
 
 /**
  * Helper class for all tests
@@ -66,6 +74,21 @@ public class TestsHelper {
     private static final int LOAD_TESTS_REQUESTS_LATENCY = parseTestSettings("load.tests.requests.latency");
 
     /** */
+    private static final int TRANSACTION_PRODUCTS_COUNT = parseTestSettings("transaction.products.count");
+
+    /** */
+    private static final int TRANSACTION_ORDERS_COUNT = parseTestSettings("transaction.orders.count");
+
+    /** */
+    private static final int ORDERS_YEAR;
+
+    /** */
+    private static final int ORDERS_MONTH;
+
+    /** */
+    private static final int ORDERS_DAY;
+
+    /** */
     private static final String LOAD_TESTS_PERSISTENCE_SETTINGS = TESTS_SETTINGS.getString("load.tests.persistence.settings");
 
     /** */
@@ -78,14 +101,30 @@ public class TestsHelper {
     private static final Generator LOAD_TESTS_VALUE_GENERATOR;
 
     /** */
-    private static int parseTestSettings(String name) {
-        return Integer.parseInt(TESTS_SETTINGS.getString(name));
-    }
+    private static final String HOST_PREFIX;
 
     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();
+
+            String[] parts = SystemHelper.HOST_IP.split("\\.");
+
+            String prefix = parts[3];
+            prefix = prefix.length() > 2 ? prefix.substring(prefix.length() - 2) : prefix;
+
+            HOST_PREFIX = prefix;
+
+            Calendar cl = Calendar.getInstance();
+
+            String year = TESTS_SETTINGS.getString("orders.year");
+            ORDERS_YEAR = !year.trim().isEmpty() ? Integer.parseInt(year) : cl.get(Calendar.YEAR);
+
+            String month = TESTS_SETTINGS.getString("orders.month");
+            ORDERS_MONTH = !month.trim().isEmpty() ? Integer.parseInt(month) : cl.get(Calendar.MONTH);
+
+            String day = TESTS_SETTINGS.getString("orders.day");
+            ORDERS_DAY = !day.trim().isEmpty() ? Integer.parseInt(day) : cl.get(Calendar.DAY_OF_MONTH);
         }
         catch (Throwable e) {
             throw new RuntimeException("Failed to initialize TestsHelper", e);
@@ -93,6 +132,11 @@ public class TestsHelper {
     }
 
     /** */
+    private static int parseTestSettings(String name) {
+        return Integer.parseInt(TESTS_SETTINGS.getString(name));
+    }
+
+    /** */
     public static int getLoadTestsThreadsCount() {
         return LOAD_TESTS_THREADS_COUNT;
     }
@@ -275,6 +319,130 @@ public class TestsHelper {
     }
 
     /** */
+    public static List<CacheEntryImpl<Long, Product>> generateProductEntries() {
+        List<CacheEntryImpl<Long, Product>> entries = new LinkedList<>();
+
+        for (long i = 0; i < BULK_OPERATION_SIZE; i++)
+            entries.add(new CacheEntryImpl<>(i, generateRandomProduct(i)));
+
+        return entries;
+    }
+
+    /** */
+    public static Collection<Long> getProductIds(Collection<CacheEntryImpl<Long, Product>> entries) {
+        List<Long> ids = new LinkedList<>();
+
+        for (CacheEntryImpl<Long, Product> entry : entries)
+            ids.add(entry.getKey());
+
+        return ids;
+    }
+
+    /** */
+    public static Map<Long, Product> generateProductsMap() {
+        return generateProductsMap(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Map<Long, Product> generateProductsMap(int count) {
+        Map<Long, Product> map = new HashMap<>();
+
+        for (long i = 0; i < count; i++)
+            map.put(i, generateRandomProduct(i));
+
+        return map;
+    }
+
+    /** */
+    public static Collection<CacheEntryImpl<Long, ProductOrder>> generateOrderEntries() {
+        Collection<CacheEntryImpl<Long, ProductOrder>> entries = new LinkedList<>();
+
+        for (long i = 0; i < BULK_OPERATION_SIZE; i++) {
+            ProductOrder order = generateRandomOrder(i);
+            entries.add(new CacheEntryImpl<>(order.getId(), order));
+        }
+
+        return entries;
+    }
+
+    /** */
+    public static Map<Long, ProductOrder> generateOrdersMap() {
+        return generateOrdersMap(BULK_OPERATION_SIZE);
+    }
+
+    /** */
+    public static Map<Long, ProductOrder> generateOrdersMap(int count) {
+        Map<Long, ProductOrder> map = new HashMap<>();
+
+        for (long i = 0; i < count; i++) {
+            ProductOrder order = generateRandomOrder(i);
+            map.put(order.getId(), order);
+        }
+
+        return map;
+    }
+
+    /** */
+    public static Map<Long, List<CacheEntryImpl<Long, ProductOrder>>> generateOrdersPerProductEntries(
+            Collection<CacheEntryImpl<Long, Product>> products) {
+        return generateOrdersPerProductEntries(products, TRANSACTION_ORDERS_COUNT);
+    }
+
+    /** */
+    public static Map<Long, List<CacheEntryImpl<Long, ProductOrder>>> generateOrdersPerProductEntries(
+            Collection<CacheEntryImpl<Long, Product>> products, int ordersPerProductCount) {
+        Map<Long, List<CacheEntryImpl<Long, ProductOrder>>> map = new HashMap<>();
+
+        for (CacheEntryImpl<Long, Product> entry : products) {
+            List<CacheEntryImpl<Long, ProductOrder>> orders = new LinkedList<>();
+
+            for (long i = 0; i < ordersPerProductCount; i++) {
+                ProductOrder order = generateRandomOrder(entry.getKey());
+                orders.add(new CacheEntryImpl<>(order.getId(), order));
+            }
+
+            map.put(entry.getKey(), orders);
+        }
+
+        return map;
+    }
+
+    /** */
+    public static Map<Long, Map<Long, ProductOrder>> generateOrdersPerProductMap(Map<Long, Product> products) {
+        return generateOrdersPerProductMap(products, TRANSACTION_ORDERS_COUNT);
+    }
+
+    /** */
+    public static Map<Long, Map<Long, ProductOrder>> generateOrdersPerProductMap(Map<Long, Product> products,
+                                                                                 int ordersPerProductCount) {
+        Map<Long, Map<Long, ProductOrder>> map = new HashMap<>();
+
+        for (Map.Entry<Long, Product> entry : products.entrySet()) {
+            Map<Long, ProductOrder> orders = new HashMap<>();
+
+            for (long i = 0; i < ordersPerProductCount; i++) {
+                ProductOrder order = generateRandomOrder(entry.getKey());
+                orders.put(order.getId(), order);
+            }
+
+            map.put(entry.getKey(), orders);
+        }
+
+        return map;
+    }
+
+    public static Collection<Long> getOrderIds(Map<Long, List<CacheEntryImpl<Long, ProductOrder>>> orders) {
+        Set<Long> ids = new HashSet<>();
+
+        for (Long key : orders.keySet()) {
+            for (CacheEntryImpl<Long, ProductOrder> entry : orders.get(key))
+                ids.add(entry.getKey());
+        }
+
+        return ids;
+    }
+
+    /** */
     public static Person generateRandomPerson(long personNum) {
         int phonesCnt = RANDOM.nextInt(4);
 
@@ -293,6 +461,33 @@ public class TestsHelper {
     }
 
     /** */
+    public static Product generateRandomProduct(long id) {
+        return new Product(id, randomString(2), randomString(6), randomString(20), generateProductPrice(id));
+    }
+
+    /** */
+    public static ProductOrder generateRandomOrder(long productId) {
+        return generateRandomOrder(productId, RANDOM.nextInt(10000));
+    }
+
+    /** */
+    private static ProductOrder generateRandomOrder(long productId, int saltedNumber) {
+        Calendar cl = Calendar.getInstance();
+        cl.set(Calendar.YEAR, ORDERS_YEAR);
+        cl.set(Calendar.MONTH, ORDERS_MONTH);
+        cl.set(Calendar.DAY_OF_MONTH, ORDERS_DAY);
+
+        long id = Long.parseLong(productId + System.currentTimeMillis() + HOST_PREFIX + saltedNumber);
+
+        return generateRandomOrder(id, productId, cl.getTime());
+    }
+
+    /** */
+    public static ProductOrder generateRandomOrder(long id, long productId, Date date) {
+        return new ProductOrder(id, productId, generateProductPrice(productId), date, 1 + RANDOM.nextInt(20));
+    }
+
+    /** */
     public static boolean checkMapsEqual(Map map1, Map map2) {
         if (map1 == null || map2 == null || map1.size() != map2.size())
             return false;
@@ -360,6 +555,66 @@ public class TestsHelper {
     }
 
     /** */
+    public static <K> boolean checkProductCollectionsEqual(Map<K, Product> map, Collection<CacheEntryImpl<K, Product>> col) {
+        if (map == null || col == null || map.size() != col.size())
+            return false;
+
+        for (CacheEntryImpl<K, Product> entry : col)
+            if (!entry.getValue().equals(map.get(entry.getKey())))
+                return false;
+
+        return true;
+    }
+
+    /** */
+    public static <K> boolean checkProductMapsEqual(Map<K, Product> map1, Map<K, Product> map2) {
+        if (map1 == null || map2 == null || map1.size() != map2.size())
+            return false;
+
+        for (K key : map1.keySet()) {
+            Product product1 = map1.get(key);
+            Product product2 = map2.get(key);
+
+            boolean equals = product1 != null && product2 != null && product1.equals(product2);
+
+            if (!equals)
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
+    public static <K> boolean checkOrderCollectionsEqual(Map<K, ProductOrder> map, Collection<CacheEntryImpl<K, ProductOrder>> col) {
+        if (map == null || col == null || map.size() != col.size())
+            return false;
+
+        for (CacheEntryImpl<K, ProductOrder> entry : col)
+            if (!entry.getValue().equals(map.get(entry.getKey())))
+                return false;
+
+        return true;
+    }
+
+    /** */
+    public static <K> boolean checkOrderMapsEqual(Map<K, ProductOrder> map1, Map<K, ProductOrder> map2) {
+        if (map1 == null || map2 == null || map1.size() != map2.size())
+            return false;
+
+        for (K key : map1.keySet()) {
+            ProductOrder order1 = map1.get(key);
+            ProductOrder order2 = map2.get(key);
+
+            boolean equals = order1 != null && order2 != null && order1.equals(order2);
+
+            if (!equals)
+                return false;
+        }
+
+        return true;
+    }
+
+    /** */
     public static String randomString(int len) {
         StringBuilder builder = new StringBuilder(len);
 
@@ -378,4 +633,28 @@ public class TestsHelper {
 
         return builder.toString();
     }
+
+    /** */
+    private static float generateProductPrice(long productId) {
+        long id = productId < 1000 ?
+                (((productId + 1) * (productId + 1) * 1000) / 2) * 10 :
+                (productId / 20) * (productId / 20);
+
+        id = id == 0 ? 24 : id;
+
+        float price = Long.parseLong(Long.toString(id).replace("0", ""));
+
+        int i = 0;
+
+        while (price > 100) {
+            if (i % 2 != 0)
+                price = price / 2;
+            else
+                price = (float) Math.sqrt(price);
+
+            i++;
+        }
+
+        return ((float)((int)(price * 100))) / 100.0F;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml b/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
index cd23a2a..c9b45c8 100644
--- a/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
+++ b/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml
@@ -46,6 +46,16 @@
         <constructor-arg type="org.springframework.core.io.Resource" value="classpath:org/apache/ignite/tests/persistence/pojo/persistence-settings-4.xml" />
     </bean>
 
+    <!-- Persistence settings for 'product' -->
+    <bean id="product_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/product.xml" />
+    </bean>
+
+    <!-- Persistence settings for 'order' -->
+    <bean id="order_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/order.xml" />
+    </bean>
+
     <!-- Ignite configuration -->
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <property name="cacheConfiguration">
@@ -89,7 +99,7 @@
                     </property>
                 </bean>
 
-                <!-- Configuring persistence for "cache3" cache -->
+                <!-- Configuring persistence for "cache4" cache -->
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="name" value="cache4"/>
                     <property name="readThrough" value="true"/>
@@ -101,6 +111,35 @@
                         </bean>
                     </property>
                 </bean>
+
+                <!-- Configuring persistence for "product" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="product"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="product_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
+                <!-- Configuring persistence for "order" cache -->
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="order"/>
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory">
+                            <property name="dataSourceBean" value="cassandraAdminDataSource"/>
+                            <property name="persistenceSettingsBean" value="order_persistence_settings"/>
+                        </bean>
+                    </property>
+                </bean>
+
             </list>
         </property>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/order.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/order.xml b/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/order.xml
new file mode 100644
index 0000000..d616364
--- /dev/null
+++ b/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/order.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="order">
+    <keyPersistence class="java.lang.Long" column="id" strategy="PRIMITIVE" />
+    <valuePersistence class="org.apache.ignite.tests.pojos.ProductOrder" strategy="POJO" />
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/product.xml
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/product.xml b/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/product.xml
new file mode 100644
index 0000000..c761e1c
--- /dev/null
+++ b/modules/cassandra/store/src/test/resources/org/apache/ignite/tests/persistence/pojo/product.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="product">
+    <keyPersistence class="java.lang.Long" column="id" strategy="PRIMITIVE" />
+    <valuePersistence class="org.apache.ignite.tests.pojos.Product" strategy="POJO" />
+</persistence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b8aca64/modules/cassandra/store/src/test/resources/tests.properties
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/test/resources/tests.properties b/modules/cassandra/store/src/test/resources/tests.properties
index 2c91e57..b11f2c8 100644
--- a/modules/cassandra/store/src/test/resources/tests.properties
+++ b/modules/cassandra/store/src/test/resources/tests.properties
@@ -16,6 +16,21 @@
 # Number of elements for CacheStore bulk operations: loadAll, writeAll, deleteAll
 bulk.operation.size=100
 
+# Number of product per transaction
+transaction.products.count=2
+
+# Number of orders per transaction
+transaction.orders.count=10
+
+# Year to use for generating new orders
+orders.year=
+
+# Month to use for generating new orders
+orders.month=
+
+# Day of month to use for generating new orders
+orders.day=
+
 # ----- Load tests settings -----
 
 # Ignite cache to be used by load tests