You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dg...@apache.org on 2019/08/29 09:16:49 UTC

[ignite] branch master updated: IGNITE-12116 Support array as key for cache

This is an automated email from the ASF dual-hosted git repository.

dgovorukhin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new e22e599  IGNITE-12116 Support array as key for cache
e22e599 is described below

commit e22e5998c5f830be97addca1b879abcacfc99bc2
Author: mstepachev <ma...@gmail.com>
AuthorDate: Thu Aug 29 12:16:08 2019 +0300

    IGNITE-12116 Support array as key for cache
    
    Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>
---
 .../cache/GridCacheDefaultAffinityKeyMapper.java   |   4 +
 .../processors/cache/KeyCacheObjectImpl.java       |   6 +-
 .../apache/ignite/internal/util/IgniteUtils.java   |  36 ++++
 .../ignite/cache/store/StoreArrayKeyTest.java      | 217 +++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java    |   3 +
 .../processors/cache/index/ArrayIndexTest.java     | 172 ++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java           |   2 +
 .../IgniteGetAllPutAllTxBytesKeyBenchmark.java     |  76 ++++++++
 .../cache/IgniteGetAndPutBytesKeyBenchmark.java    |  46 +++++
 .../cache/IgniteGetAndPutTxBytesKeyBenchmark.java  |  73 +++++++
 .../cache/IgnitePutBytesKeyBenchmark.java          |  46 +++++
 11 files changed, 679 insertions(+), 2 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
index 8d84254..b9fd67b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
@@ -27,6 +27,7 @@ import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.GridReflectionCache;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -77,6 +78,9 @@ public class GridCacheDefaultAffinityKeyMapper implements AffinityKeyMapper {
     @Override public Object affinityKey(Object key) {
         GridArgumentCheck.notNull(key, "key");
 
+        if(key.getClass().isArray())
+            return IgniteUtils.hashCode(key);
+
         try {
             Object o = reflectCache.firstFieldValue(key);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
index 92e8a4f..89e4338 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheObjectImpl.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.nio.ByteBuffer;
+import java.util.Objects;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -113,7 +115,7 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
     @Override public int hashCode() {
         assert val != null;
 
-        return val.hashCode();
+        return IgniteUtils.hashCode(val);
     }
 
     /** {@inheritDoc} */
@@ -198,7 +200,7 @@ public class KeyCacheObjectImpl extends CacheObjectAdapter implements KeyCacheOb
 
         KeyCacheObjectImpl other = (KeyCacheObjectImpl)obj;
 
-        return val.equals(other.val);
+        return Objects.deepEquals(val, other.val);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index a445764..01c55b2 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -5359,6 +5359,42 @@ public abstract class IgniteUtils {
         return map;
     }
 
+
+    /**
+     * Calculate a hashCode for an array.
+     *
+     * @param obj Object.
+     */
+    public static int hashCode(Object obj) {
+        if (obj.getClass().isArray()) {
+            if (obj instanceof byte[])
+                return Arrays.hashCode((byte[])obj);
+            if (obj instanceof short[])
+                return Arrays.hashCode((short[])obj);
+            if (obj instanceof int[])
+                return Arrays.hashCode((int[])obj);
+            if (obj instanceof long[])
+                return Arrays.hashCode((long[])obj);
+            if (obj instanceof float[])
+                return Arrays.hashCode((float[])obj);
+            if (obj instanceof double[])
+                return Arrays.hashCode((double[])obj);
+            if (obj instanceof char[])
+                return Arrays.hashCode((char[])obj);
+            if (obj instanceof boolean[])
+                return Arrays.hashCode((boolean[])obj);
+
+            int result = 1;
+
+            for (Object element : (Object[])obj)
+                result = 31 * result + hashCode(element);
+
+            return result;
+        }
+        else
+            return obj.hashCode();
+    }
+
     /**
      * @param in Input.
      * @return Read map.
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/StoreArrayKeyTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/StoreArrayKeyTest.java
new file mode 100644
index 0000000..26eaf6b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/StoreArrayKeyTest.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.cache.store;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Checks that array keys are supported.
+ */
+@RunWith(Parameterized.class)
+public class StoreArrayKeyTest extends GridCommonAbstractTest {
+    /** Cache. */
+    private static final String CACHE = "cache-1";
+
+    /** Cache with backups. */
+    private static final String CACHE_WITH_BACKUPS = "cache-with-backup";
+
+    /** Node 1. */
+    private IgniteEx node1;
+
+    /** Node 2. */
+    private IgniteEx node2;
+
+    /** First key. */
+    @Parameterized.Parameter(0)
+    public Object firstKey;
+
+    /** Second key. */
+    @Parameterized.Parameter(1)
+    public Object secondKey;
+
+    /** Like first key but other object. */
+    @Parameterized.Parameter(2)
+    public Object likeFirstKey;
+
+    /**
+     *
+     */
+    @Parameterized.Parameters()
+    public static Collection<Object[]> dataset() {
+        return Arrays.asList(
+            new byte[][] {new byte[] {1, 2, 3}, new byte[] {3, 2, 1}, new byte[] {1, 2, 3}},
+            new short[][] {new short[] {1, 2, 3}, new short[] {3, 2, 1}, new short[] {1, 2, 3}},
+            new int[][] {new int[] {1, 2, 3}, new int[] {3, 2, 1}, new int[] {1, 2, 3}},
+            new long[][] {new long[] {1, 2, 3}, new long[] {3, 2, 1}, new long[] {1, 2, 3}},
+            new float[][] {new float[] {1, 2, 3}, new float[] {3, 2, 1}, new float[] {1, 2, 3}},
+            new double[][] {new double[] {1, 2, 3}, new double[] {3, 2, 1}, new double[] {1, 2, 3}},
+            new char[][] {new char[] {1, 2, 3}, new char[] {3, 2, 1}, new char[] {1, 2, 3}},
+            new boolean[][] {new boolean[] {true, false, true}, new boolean[] {false, true, false}, new boolean[] {true, false, true}},
+            new String[][] {new String[] {"a", "b", "c"}, new String[] {"c", "b", "a"}, new String[] {"a", "b", "c"}},
+            new Object[][] {
+                new String[][] {
+                    new String[] {"a", "b", "c"},
+                    new String[] {"a", "b", "c"},
+                    new String[] {"a", "b", "c"}
+                },
+                new String[] {"c", "b", "a"},
+                new String[][] {
+                    new String[] {"a", "b", "c"},
+                    new String[] {"a", "b", "c"},
+                    new String[] {"a", "b", "c"}
+                }}
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDataRegionConfigurations(new DataRegionConfiguration()
+                    .setName("pdr")
+                    .setPersistenceEnabled(true))
+        );
+
+        cfg.setCacheConfiguration(
+            new CacheConfiguration(CACHE_WITH_BACKUPS)
+                .setDataRegionName("pdr")
+                .setBackups(1)
+                .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC),
+            new CacheConfiguration(CACHE)
+                .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC)
+        );
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    @Before
+    public void setUp() throws Exception {
+        node1 = startGrid(0);
+        node2 = startGrid(1);
+
+        node1.cluster().active(true);
+    }
+
+    /**
+     *
+     */
+    @After
+    public void tearDown() throws Exception {
+        stopAllGrids();
+        cleanPersistenceDir();
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldReadWriteKey() {
+        IgniteCache<Object, Object> cache = node1.getOrCreateCache(CACHE);
+
+        cache.put(firstKey, 1);
+        cache.put(secondKey, 2);
+        cache.put(likeFirstKey, 3);
+
+        assertEquals(2, cache.size());
+
+        assertEquals(3, cache.get(likeFirstKey));
+        assertEquals(3, cache.get(firstKey));
+        assertEquals(2, cache.get(secondKey));
+
+        assertTrue(cache.containsKey(likeFirstKey));
+        assertTrue(cache.containsKey(firstKey));
+        assertTrue(cache.containsKey(secondKey));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldRemoveBySameKey() {
+        IgniteCache<Object, Object> cache = node1.getOrCreateCache(CACHE);
+
+        cache.put(firstKey, 1);
+        cache.put(secondKey, 2);
+        cache.put(likeFirstKey, 3);
+
+        assertEquals(2, cache.size());
+
+        cache.remove(firstKey);
+        cache.remove(secondKey);
+
+        assertEquals(0, cache.size());
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldRemoveAllCache() {
+        IgniteCache<Object, Object> cache = node1.getOrCreateCache(CACHE);
+
+        cache.put(firstKey, "val");
+
+        cache.removeAll();
+
+        assertEquals(0, cache.size());
+
+        Iterator<Cache.Entry<Object, Object>> it = cache.iterator();
+
+        assertFalse(it.hasNext());
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldClearCache() {
+        IgniteCache<Object, Object> cache = node1.getOrCreateCache(CACHE);
+
+        cache.put(firstKey, "val");
+
+        cache.clear();
+
+        assertEquals(0, cache.size());
+
+        Iterator<Cache.Entry<Object, Object>> it = cache.iterator();
+
+        assertFalse(it.hasNext());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 2683184..304565c 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -28,6 +28,7 @@ import org.apache.ignite.cache.store.CacheTransactionalStoreReadFromBackupTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
 import org.apache.ignite.cache.store.GridStoreLoadCacheTest;
+import org.apache.ignite.cache.store.StoreArrayKeyTest;
 import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerStoreKeepBinarySelfTest;
@@ -216,6 +217,8 @@ public class IgniteCacheTestSuite {
         GridTestUtils.addTestIfNeeded(suite, IgniteCachePrivateExecutionContextTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IgniteCacheSharedExecutionContextTest.class, ignoredTests);
 
+        GridTestUtils.addTestIfNeeded(suite, StoreArrayKeyTest.class, ignoredTests);
+
         // Warmup closure tests.
         GridTestUtils.addTestIfNeeded(suite, IgniteWarmupClosureSelfTest.class, ignoredTests);
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ArrayIndexTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ArrayIndexTest.java
new file mode 100644
index 0000000..1225de6
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ArrayIndexTest.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.cache.index;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Checks that sql operation works by arrays.
+ */
+public class ArrayIndexTest extends AbstractIndexingCommonTest {
+    /**
+     * @throws Exception if fails.
+     */
+    @After
+    public void tearDown() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * @throws Exception if fails.
+     */
+    @Before
+    public void setUp() throws Exception {
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration().setPersistenceEnabled(true)
+            )
+        );
+
+        cfg.setCacheConfiguration(
+            new CacheConfiguration(DEFAULT_CACHE_NAME)
+                .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC)
+                .setQueryEntities(Collections.singleton(
+                    new QueryEntity()
+                        .setKeyType(byte[].class.getName())
+                        .setValueType(Integer.class.getName())
+                ))
+        );
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldSelectAllRows() throws Exception {
+        IgniteEx ex = startGrid(0);
+
+        ex.cluster().active(true);
+
+        IgniteCache<Object, Object> cache = ex.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        cache.put(new byte[] {8, 9, 10}, 3);
+        cache.put(new byte[] {1, 2, 3}, 1);
+        cache.put(new byte[] {5, 6, 7}, 2);
+
+        List<List<?>> sorted = cache.query(new SqlFieldsQuery("select _key, _val from Integer")).getAll();
+
+        assertEquals(3, sorted.size());
+
+        List<?> first = sorted.get(0);
+        assertTrue(Objects.deepEquals(first.get(0), new byte[] {1, 2, 3}));
+        assertTrue(Objects.deepEquals(first.get(1), 1));
+
+        List<?> second = sorted.get(1);
+        assertTrue(Objects.deepEquals(second.get(0), new byte[] {5, 6, 7}));
+        assertTrue(Objects.deepEquals(second.get(1), 2));
+
+        List<?> third = sorted.get(2);
+        assertTrue(Objects.deepEquals(third.get(0), new byte[] {8, 9, 10}));
+        assertTrue(Objects.deepEquals(third.get(1), 3));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldSelectParticularValue() throws Exception {
+        IgniteEx ex = startGrid(0);
+
+        ex.cluster().active(true);
+
+        IgniteCache<Object, Object> cache = ex.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        cache.put(new byte[] {1, 2, 3}, 1);
+        cache.put(new byte[] {5, 6, 7}, 2);
+        cache.put(new byte[] {8, 9, 10}, 3);
+
+        List<List<?>> res = cache.query(new SqlFieldsQuery("select _key, _val from Integer where _key = ?")
+            .setArgs(new byte[] {5, 6, 7}))
+            .getAll();
+
+        assertEquals(1, res.size());
+
+        List<?> row = res.get(0);
+        assertTrue(Objects.deepEquals(row.get(0), new byte[] {5, 6, 7}));
+        assertTrue(Objects.deepEquals(row.get(1), 2));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void shouldCreateTableWithBinaryKey() throws Exception {
+        byte[] key = {1, 2, 3, 4};
+
+        IgniteEx ex = startGrid(0);
+
+        ex.cluster().active(true);
+
+        executeSql(ex, "CREATE TABLE Binary_Entries (key binary(16) not null, val binary(16), PRIMARY KEY(key))");
+
+        executeSql(ex, "INSERT INTO Binary_Entries(key, val) VALUES (x'" + Hex.encodeHexString(key) + "', x'01020304')");
+
+        assertEquals(ex.cache("SQL_PUBLIC_BINARY_ENTRIES").size(), 1);
+        assertTrue(ex.cache("SQL_PUBLIC_BINARY_ENTRIES").containsKey(key));
+
+    }
+
+    /**
+     *
+     */
+    private List<List<?>> executeSql(IgniteEx node, String sqlText) throws Exception {
+        GridQueryProcessor qryProc = node.context().query();
+
+        return qryProc.querySqlFields(new SqlFieldsQuery(sqlText), true).getAll();
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 17e8ff1..c689c98 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -120,6 +120,7 @@ import org.apache.ignite.internal.processors.cache.distributed.replicated.Ignite
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.encryption.EncryptedSqlTableTest;
+import org.apache.ignite.internal.processors.cache.index.ArrayIndexTest;
 import org.apache.ignite.internal.processors.cache.index.BasicIndexMultinodeTest;
 import org.apache.ignite.internal.processors.cache.index.BasicIndexTest;
 import org.apache.ignite.internal.processors.cache.index.ComplexPrimaryKeyUnwrapSelfTest;
@@ -288,6 +289,7 @@ import org.junit.runners.Suite;
     MultipleStatementsSqlQuerySelfTest.class,
 
     BasicIndexTest.class,
+    ArrayIndexTest.class,
     BasicIndexMultinodeTest.class,
 
     // Misc tests.
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBytesKeyBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBytesKeyBenchmark.java
new file mode 100644
index 0000000..893d4e4
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAllPutAllTxBytesKeyBenchmark.java
@@ -0,0 +1,76 @@
+/*
+ * 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.Comparator;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+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 IgniteGetAllPutAllTxBytesKeyBenchmark extends IgniteCacheAbstractBenchmark<byte[], 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(), () -> {
+            SortedMap<byte[], Integer> vals = new TreeMap<>(Comparator.comparing(String::new));
+
+            for (int i = 0; i < args.batch(); i++) {
+                int base = r.nextRandom();
+
+                byte[] key = String.valueOf(base).getBytes();
+
+                vals.put(key, base);
+            }
+
+            IgniteCache<byte[], Integer> cache = cacheForOperation();
+
+            cache.getAll(vals.keySet());
+
+            cache.putAll(vals);
+
+            return null;
+        });
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<byte[], Integer> cache() {
+        return ignite().cache("tx");
+    }
+}
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBytesKeyBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBytesKeyBenchmark.java
new file mode 100644
index 0000000..16443a8
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutBytesKeyBenchmark.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+
+/**
+ * Ignite benchmark that performs getAndPut operations.
+ */
+public class IgniteGetAndPutBytesKeyBenchmark extends IgniteCacheAbstractBenchmark<byte[], Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int base = nextRandom(args.range());
+
+        byte[] key = String.valueOf(base).getBytes();
+
+        IgniteCache<byte[], Object> cache = cacheForOperation();
+
+        cache.getAndPut(key, new SampleValue(base));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<byte[], Object> cache() {
+        return ignite().cache("atomic");
+    }
+}
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBytesKeyBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBytesKeyBenchmark.java
new file mode 100644
index 0000000..f774e54
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetAndPutTxBytesKeyBenchmark.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.IgniteSystemProperties;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.yardstick.IgniteBenchmarkUtils;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+/**
+ * Ignite benchmark that performs getAndPut operations.
+ */
+public class IgniteGetAndPutTxBytesKeyBenchmark extends IgniteCacheAbstractBenchmark<byte[], Object> {
+    /** */
+    private IgniteTransactions txs;
+
+    /** */
+    private Callable<Void> clo;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK"))
+            ignite().compute().broadcast(new WaitMapExchangeFinishCallable());
+
+        txs = ignite().transactions();
+
+        clo = () -> {
+            int base = nextRandom(args.range());
+
+            byte[] key = String.valueOf(base).getBytes();
+
+            IgniteCache<byte[], Object> cache = cacheForOperation();
+
+            cache.getAndPut(key, new SampleValue(base));
+
+            return null;
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        IgniteBenchmarkUtils.doInTransaction(txs, args.txConcurrency(), args.txIsolation(), clo);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<byte[], Object> cache() {
+        return ignite().cache("tx");
+    }
+}
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBytesKeyBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBytesKeyBenchmark.java
new file mode 100644
index 0000000..3d0ae42
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBytesKeyBenchmark.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+
+/**
+ * Ignite benchmark that performs put operations.
+ */
+public class IgnitePutBytesKeyBenchmark extends IgniteCacheAbstractBenchmark<byte[], Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int base = nextRandom(args.range());
+
+        byte[] key = String.valueOf(base).getBytes();
+
+        IgniteCache<byte[], Object> cache = cacheForOperation();
+
+        cache.put(key, new SampleValue(base));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<byte[], Object> cache() {
+        return ignite().cache("atomic");
+    }
+}