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

[16/50] [abbrv] ignite git commit: 2224

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 93ff515..3bd87cf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -53,6 +53,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
@@ -556,6 +557,30 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception In case of error.
      */
+    public void testGetEntry() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        CacheEntry<String, Integer> key1e =  cache.getEntry("key1");
+        CacheEntry<String, Integer> key2e =  cache.getEntry("key2");
+        CacheEntry<String, Integer> wrongKeye =  cache.getEntry("wrongKey");
+
+        assert key1e.getValue() == 1;
+        assert key1e.getKey().equals("key1");
+        assert key1e.version() != null;
+
+        assert key2e.getValue() == 2;
+        assert key2e.getKey().equals("key2");
+        assert key2e.version() != null;
+
+        assert wrongKeye == null;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
     public void testGetAsync() throws Exception {
         IgniteCache<String, Integer> cache = jcache();
 
@@ -664,6 +689,122 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception In case of error.
      */
+    public void testGetEntries() throws Exception {
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        final IgniteCache<String, Integer> cache = jcache();
+
+        try {
+            cache.put("key1", 1);
+            cache.put("key2", 2);
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getEntries(null).isEmpty();
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        assert cache.getEntries(Collections.<String>emptySet()).isEmpty();
+
+        Collection<CacheEntry<String, Integer>> c1 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+        info("Retrieved c1: " + c1);
+
+        assert 2 == c1.size() : "Invalid collection: " + c1;
+
+        boolean b1 = false;
+        boolean b2 = false;
+
+        for (CacheEntry<String, Integer> e: c1){
+            if (e.getKey().equals("key1") && e.getValue().equals(1))
+                b1 = true;
+
+            if (e.getKey().equals("key2") && e.getValue().equals(2))
+                b2 = true;
+        }
+
+        assertTrue(b1 && b2);
+
+        Collection<CacheEntry<String, Integer>> c2 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+        info("Retrieved c2: " + c2);
+
+        assert 2 == c2.size() : "Invalid collection: " + c2;
+
+        b1 = false;
+        b2 = false;
+
+        for (CacheEntry<String, Integer> e: c2){
+            if (e.getKey().equals("key1") && e.getValue().equals(1))
+                b1 = true;
+
+            if (e.getKey().equals("key2") && e.getValue().equals(2))
+                b2 = true;
+        }
+
+        assertTrue(b1 && b2);
+
+        // Now do the same checks but within transaction.
+        if (txShouldBeUsed()) {
+            try (Transaction tx0 = transactions().txStart()) {
+                assert cache.getEntries(Collections.<String>emptySet()).isEmpty();
+
+                c1 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+                info("Retrieved c1: " + c1);
+
+                assert 2 == c1.size() : "Invalid collection: " + c1;
+
+                b1 = false;
+                b2 = false;
+
+                for (CacheEntry<String, Integer> e : c1) {
+                    if (e.getKey().equals("key1") && e.getValue().equals(1))
+                        b1 = true;
+
+                    if (e.getKey().equals("key2") && e.getValue().equals(2))
+                        b2 = true;
+                }
+
+                assertTrue(b1 && b2);
+
+                c2 = cache.getEntries(ImmutableSet.of("key1", "key2", "key9999"));
+
+                info("Retrieved c2: " + c2);
+
+                assert 2 == c2.size() : "Invalid collection: " + c2;
+
+                b1 = false;
+                b2 = false;
+
+                for (CacheEntry<String, Integer> e : c2) {
+                    if (e.getKey().equals("key1") && e.getValue().equals(1))
+                        b1 = true;
+
+                    if (e.getKey().equals("key2") && e.getValue().equals(2))
+                        b2 = true;
+                }
+
+                assertTrue(b1 && b2);
+
+                tx0.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
     public void testGetAllWithNulls() throws Exception {
         final IgniteCache<String, Integer> cache = jcache();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
index c57869f..f50a3e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheInterceptorAbstractSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -29,6 +30,7 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
@@ -152,19 +154,32 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
      * @throws Exception If failed.
      */
     public void testGet() throws Exception {
-        testGet(primaryKey(0));
+        testGet(primaryKey(0), false);
 
         afterTest();
 
         if (cacheMode() != LOCAL)
-            testGet(backupKey(0));
+            testGet(backupKey(0), false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetEntry() throws Exception {
+        testGet(primaryKey(0), true);
+
+        afterTest();
+
+        if (cacheMode() != LOCAL)
+            testGet(backupKey(0), true);
     }
 
     /**
      * @param key Key.
+     * @param needVer Need version.
      * @throws Exception If failed.
      */
-    private void testGet(String key) throws Exception {
+    private void testGet(String key, boolean needVer) throws Exception {
         // Try when value is not in cache.
 
         interceptor.retInterceptor = new NullGetInterceptor();
@@ -173,7 +188,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         IgniteCache<String, Integer> cache = jcache(0);
 
-        assertEquals(null, cache.get(key));
+        assertEquals(null, needVer ? cache.getEntry(key) : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -185,7 +200,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         log.info("Get 2.");
 
-        assertEquals((Integer)1, cache.get(key));
+        assertEquals((Integer)1, needVer ? cache.getEntry(key).getValue() : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -207,7 +222,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         log.info("Get 3.");
 
-        assertEquals(null, cache.get(key));
+        assertEquals(null, needVer ? cache.getEntry(key) : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -223,7 +238,7 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         log.info("Get 4.");
 
-        assertEquals((Integer)101, cache.get(key));
+        assertEquals((Integer)101, needVer ? cache.getEntry(key).getValue() : cache.get(key));
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -241,9 +256,16 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         IgniteCache<String, Integer> cacheAsync = cache.withAsync();
 
-        cacheAsync.get(key);
+        if (needVer) {
+            cacheAsync.getEntry(key);
 
-        assertEquals((Integer)101, cacheAsync.<Integer>future().get());
+            assertEquals((Integer)101, cacheAsync.<CacheEntry<String, Integer>>future().get().getValue());
+        }
+        else {
+            cacheAsync.get(key);
+
+            assertEquals((Integer)101, cacheAsync.<Integer>future().get());
+        }
 
         assertEquals(1, interceptor.invokeCnt.get());
 
@@ -258,6 +280,20 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
      * @throws Exception If failed.
      */
     public void testGetAll() throws Exception {
+        testGetAll(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetEntries() throws Exception {
+        testGetAll(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testGetAll(boolean needVer) throws Exception {
         Set<String> keys = new LinkedHashSet<>();
 
         for (int i = 0; i < 1000; i++)
@@ -269,10 +305,19 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         IgniteCache<String, Integer> cacheAsync = cache.withAsync();
 
-        Map<String, Integer> map = cache.getAll(keys);
+        Collection<CacheEntry<String, Integer>> c;
+        Map<String, Integer> map;
+
+        if (needVer){
+            c = cache.getEntries(keys);
 
-        for (String key : keys)
-            assertEquals(null, map.get(key));
+            assertTrue(c.isEmpty());
+        }else {
+            map = cache.getAll(keys);
+
+            for (String key : keys)
+                assertEquals(null, map.get(key));
+        }
 
         assertEquals(1000, interceptor.invokeCnt.get());
 
@@ -280,15 +325,28 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
         interceptor.retInterceptor = new GetAllInterceptor1();
 
-        map = cache.getAll(keys);
+        if (needVer) {
+            c = cache.getEntries(keys);
 
-        for (String key : keys) {
-            int k = Integer.valueOf(key);
+            assertEquals(500, c.size());
 
-            if (k % 2 == 0)
-                assertEquals(null, map.get(key));
-            else
-                assertEquals((Integer)(k * 2), map.get(key));
+            for (CacheEntry<String, Integer> e : c) {
+                int k = Integer.valueOf(e.getKey());
+
+                assertEquals((Integer)(k * 2), e.getValue());
+            }
+        }
+        else {
+            map = cache.getAll(keys);
+
+            for (String key : keys) {
+                int k = Integer.valueOf(key);
+
+                if (k % 2 == 0)
+                    assertEquals(null, map.get(key));
+                else
+                    assertEquals((Integer)(k * 2), map.get(key));
+            }
         }
 
         assertEquals(1000, interceptor.invokeCnt.get());
@@ -307,40 +365,72 @@ public abstract class GridCacheInterceptorAbstractSelfTest extends GridCacheAbst
 
             interceptor.retInterceptor = new GetAllInterceptor2();
 
-            if (j == 0)
-                map = cache.getAll(keys);
-            else {
-                cacheAsync.getAll(keys);
+            if (needVer) {
+                if (j == 0)
+                    c = cache.getEntries(keys);
+                else {
+                    cacheAsync.getEntries(keys);
+
+                    c = cacheAsync.<Collection<CacheEntry<String, Integer>>>future().get();
+                }
+
+                for (CacheEntry<String, Integer> e : c) {
+                    int k = Integer.valueOf(e.getKey());
+
+                    switch (k % 3) {
+                        case 1:
+                            Integer exp = k < 500 ? k : null;
 
-                map = cacheAsync.<Map<String, Integer>>future().get();
+                            assertEquals(exp, e.getValue());
+
+                            break;
+
+                        case 2:
+                            assertEquals((Integer)(k * 3), e.getValue());
+
+                            break;
+
+                        default:
+                            fail();
+                    }
+                }
             }
+            else {
+                if (j == 0)
+                    map = cache.getAll(keys);
+                else {
+                    cacheAsync.getAll(keys);
 
-            int i = 0;
+                    map = cacheAsync.<Map<String, Integer>>future().get();
+                }
 
-            for (String key : keys) {
-                switch (i % 3) {
-                    case 0:
-                        assertEquals(null, map.get(key));
+                int i = 0;
 
-                        break;
+                for (String key : keys) {
+                    switch (i % 3) {
+                        case 0:
+                            assertEquals(null, map.get(key));
 
-                    case 1:
-                        Integer exp = i < 500 ? i : null;
+                            break;
 
-                        assertEquals(exp, map.get(key));
+                        case 1:
+                            Integer exp = i < 500 ? i : null;
 
-                        break;
+                            assertEquals(exp, map.get(key));
 
-                    case 2:
-                        assertEquals((Integer)(i * 3), map.get(key));
+                            break;
 
-                        break;
+                        case 2:
+                            assertEquals((Integer)(i * 3), map.get(key));
 
-                    default:
-                        fail();
-                }
+                            break;
 
-                i++;
+                        default:
+                            fail();
+                    }
+
+                    i++;
+                }
             }
 
             assertEquals(1000, interceptor.invokeCnt.get());

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index 6a7416e..293ba1e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -263,7 +263,7 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
             waitForLocalEvent(grid(primary).events(), nodeEvent(primary.id()), EVT_CACHE_ENTRY_EVICTED);
 
         // Get value on other node, it should be loaded to near cache.
-        assertEquals(val, nearOther.get(key, true));
+        assertEquals(val, nearOther.get(key, true, false));
 
         entryPrimary = (GridDhtCacheEntry)dhtPrimary.peekEx(key);
         entryBackup = (GridDhtCacheEntry)dhtBackup.peekEx(key);

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index 035f1b0..7286fb3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -34,6 +34,7 @@ import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
@@ -129,7 +130,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) 
+    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args)
         throws CacheException {
         throw new UnsupportedOperationException("Method should be supported.");
     }
@@ -224,11 +225,21 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public CacheEntry<K, V> getEntry(K key) {
+        return compute.call(new GetEntryTask<K, V>(cacheName, isAsync, key));
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAll(Set<? extends K> keys) {
         return compute.call(new GetAllTask<K, V>(cacheName, isAsync, keys));
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<CacheEntry<K, V>> getEntries(Set<? extends K> keys) {
+        return compute.call(new GetEntriesTask<K, V>(cacheName, isAsync, keys));
+    }
+
+    /** {@inheritDoc} */
     @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
         return compute.call(new GetAllOutTxTask<K, V>(cacheName, isAsync, keys));
     }
@@ -710,6 +721,29 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     /**
      *
      */
+    private static class GetEntryTask<K, V> extends CacheTaskAdapter<K, V, CacheEntry<K, V>> {
+        /** Key. */
+        private final K key;
+
+        /**
+         * @param cacheName Cache name.
+         * @param async Async.
+         * @param key Key.
+         */
+        public GetEntryTask(String cacheName, boolean async, K key) {
+            super(cacheName, async);
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public CacheEntry<K, V> call() throws Exception {
+            return cache().getEntry(key);
+        }
+    }
+
+    /**
+     *
+     */
     private static class RemoveAllTask<K, V> extends CacheTaskAdapter<K, V, Void> {
         /**
          * @param cacheName Cache name.
@@ -973,6 +1007,29 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     /**
      *
      */
+    private static class GetEntriesTask<K, V> extends CacheTaskAdapter<K, V, Collection<CacheEntry<K, V>> > {
+        /** Keys. */
+        private final Set<? extends K> keys;
+
+        /**
+         * @param cacheName Cache name.
+         * @param async Async.
+         * @param keys Keys.
+         */
+        public GetEntriesTask(String cacheName, boolean async, Set<? extends K> keys) {
+            super(cacheName, async);
+            this.keys = keys;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<CacheEntry<K, V>>  call() throws Exception {
+            return cache().getEntries(keys);
+        }
+    }
+
+    /**
+     *
+     */
     private static class GetAllOutTxTask<K, V> extends CacheTaskAdapter<K, V, Map<K, V>> {
         /** Keys. */
         private final Set<? extends K> keys;

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 68e52df..45679dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -19,6 +19,12 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticReadCommittedSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticRepeatableReadSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticSerializableSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticReadCommittedSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticRepeatableReadSeltTest;
+import org.apache.ignite.internal.processors.cache.CacheGetEntryPessimisticSerializableSeltTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
 import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
@@ -241,6 +247,12 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteSystemCacheOnClientTest.class);
 
         suite.addTestSuite(CacheRemoveAllSelfTest.class);
+        suite.addTestSuite(CacheGetEntryOptimisticReadCommittedSeltTest.class);
+        suite.addTestSuite(CacheGetEntryOptimisticRepeatableReadSeltTest.class);
+        suite.addTestSuite(CacheGetEntryOptimisticSerializableSeltTest.class);
+        suite.addTestSuite(CacheGetEntryPessimisticReadCommittedSeltTest.class);
+        suite.addTestSuite(CacheGetEntryPessimisticRepeatableReadSeltTest.class);
+        suite.addTestSuite(CacheGetEntryPessimisticSerializableSeltTest.class);
 
         suite.addTestSuite(CacheStopAndDestroySelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index c638d94..0428c73 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -89,10 +89,14 @@ j=10
 CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutBenchmark -sn IgniteNode -ds ${ver}atomic-put-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutGetBenchmark -sn IgniteNode -ds ${ver}atomic-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutGetEntryBenchmark -sn IgniteNode -ds ${ver}atomic-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutTxImplicitBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-put-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-optim-repRead-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -dn IgnitePutGetEntryTxBenchmark -sn IgniteNode -ds ${ver}tx-optim-repRead-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc PESSIMISTIC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-pessim-repRead-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc PESSIMISTIC -dn IgnitePutGetEntryTxBenchmark -sn IgniteNode -ds ${ver}tx-pessim-repRead-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -txi SERIALIZABLE -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serial-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -txc OPTIMISTIC -txi SERIALIZABLE -dn IgnitePutGetEntryTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serial-put-getEntry-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds ${ver}sql-query-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds ${ver}sql-query-join-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds ${ver}sql-query-put-1-backup,\
@@ -114,6 +118,9 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-putAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllSerializableTxBenchmark -sn IgniteNode -ds ${ver}tx-putAllSerializable-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC  -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-optimistic-getEntriesPutAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-pessimistic-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc PESSIMISTIC -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-pessimistic-getEntriesPutAll-1-backup,\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetAllPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serializable-getAllPutAll-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 10 -txc OPTIMISTIC -txi SERIALIZABLE -dn IgniteGetEntriesPutAllTxBenchmark -sn IgniteNode -ds ${ver}tx-opt-serializable-getEntriesPutAll-1-backup,\
 "

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java
new file mode 100644
index 0000000..501e12d
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetEntriesPutAllTxBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
+
+/**
+ * Ignite benchmark that performs transactional putAll operations.
+ */
+public class IgniteGetEntriesPutAllTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Integer> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        final ThreadRange r = threadRange();
+
+        doInTransaction(txs, args.txConcurrency(), args.txIsolation(), new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                SortedMap<Integer, Integer> vals = new TreeMap<>();
+
+                for (int i = 0; i < args.batch(); i++) {
+                    int key = r.nextRandom();
+
+                    vals.put(key, key);
+                }
+
+                cache.getEntries(vals.keySet());
+
+                cache.putAll(vals);
+
+                return null;
+            }
+        });
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Integer> cache() {
+        return ignite().cache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java
new file mode 100644
index 0000000..1289fa1
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryBenchmark.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+
+/**
+ * Ignite benchmark that performs put and get operations.
+ */
+public class IgnitePutGetEntryBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        CacheEntry<Integer, Object> val = cache.getEntry(key);
+
+        if (val != null)
+            key = nextRandom(args.range());
+
+        cache.put(key, new SampleValue(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/01135066/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java
new file mode 100644
index 0000000..6e58b41
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetEntryTxBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.Map;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.apache.ignite.yardstick.IgniteBenchmarkUtils.doInTransaction;
+
+/**
+ * Ignite benchmark that performs transactional put and get operations.
+ */
+public class IgnitePutGetEntryTxBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** */
+    private Callable<Void> clo;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        txs = ignite().transactions();
+
+        clo = new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                int key = nextRandom(0, args.range() / 2);
+
+                CacheEntry<Integer, Object> val = cache.getEntry(key);
+
+                if (val != null)
+                    key = nextRandom(args.range() / 2, args.range());
+
+                cache.put(key, new SampleValue(key));
+
+                return null;
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("tx");
+    }
+}