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

[2/8] ignite git commit: IGNITE-2294: Implemented DML.

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index d13af12..537ccdf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -40,8 +40,11 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Query;
+import org.h2.command.dml.Update;
 import org.h2.engine.Session;
+import org.h2.expression.Expression;
 import org.h2.jdbc.JdbcConnection;
+import org.h2.util.StringUtils;
 
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -106,7 +109,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testAllExamples() throws Exception {
+    public void testParseSelectAndUnion() throws Exception {
         checkQuery("select 42");
         checkQuery("select ()");
         checkQuery("select (1)");
@@ -120,6 +123,8 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         checkQuery("select (select 1)");
         checkQuery("select (select 1, select ?)");
         checkQuery("select ((select 1), select ? + ?)");
+        checkQuery("select CURRENT_DATE");
+        checkQuery("select CURRENT_DATE()");
 
         checkQuery("select extract(year from ?)");
         checkQuery("select convert(?, timestamp)");
@@ -263,6 +268,106 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
         checkQuery("(select 2 a) union all (select 1) order by a desc nulls first limit ? offset ?");
     }
 
+    /** */
+    public void testParseMerge() throws Exception {
+        /* Plain rows w/functions, operators, defaults, and placeholders. */
+        checkQuery("merge into Person(old, name) values(5, 'John')");
+        checkQuery("merge into Person(name) values(DEFAULT)");
+        checkQuery("merge into Person(name) values(DEFAULT), (null)");
+        checkQuery("merge into Person(name, parentName) values(DEFAULT, null), (?, ?)");
+        checkQuery("merge into Person(old, name) values(5, 'John',), (6, 'Jack')");
+        checkQuery("merge into Person(old, name) values(5 * 3, DEFAULT,)");
+        checkQuery("merge into Person(old, name) values(ABS(-8), 'Max')");
+        checkQuery("merge into Person(old, name) values(5, 'Jane'), (DEFAULT, DEFAULT), (6, 'Jill')");
+        checkQuery("merge into Person(old, name, parentName) values(8 * 7, DEFAULT, 'Unknown')");
+        checkQuery("merge into Person(old, name, parentName) values" +
+            "(2016 - 1828, CONCAT('Leo', 'Tolstoy'), CONCAT(?, 'Tolstoy'))," +
+            "(?, 'AlexanderPushkin', null)," +
+            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), DEFAULT)");
+        checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
+            "('20160112', 1233, 'Ivan Ivanov', 'Peter Ivanov', 123)");
+        checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
+            "(CURRENT_DATE(), RAND(), ASCII('Hi'), INSERT('Leo Tolstoy', 4, 4, 'Max'), ASCII('HI'))");
+        checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
+            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, DEFAULT, DEFAULT)");
+        checkQuery("merge into Person(old, name) select ASCII(parentName), INSERT(parentName, 4, 4, 'Max') from " +
+            "Person where date='20110312'");
+
+        /* Subqueries. */
+        checkQuery("merge into Person(old, name) select old, parentName from Person");
+        checkQuery("merge into Person(old, name) select old, parentName from Person where old > 5");
+        checkQuery("merge into Person(old, name) select 5, 'John'");
+        checkQuery("merge into Person(old, name) select p1.old, 'Name' from person p1 join person p2 on " +
+            "p2.name = p1.parentName where p2.old > 30");
+        checkQuery("merge into Person(old) select 5 from Person UNION select street from Address limit ? offset ?");
+    }
+
+    /** */
+    public void testParseInsert() throws Exception {
+        /* Plain rows w/functions, operators, defaults, and placeholders. */
+        checkQuery("insert into Person(old, name) values(5, 'John')");
+        checkQuery("insert into Person(name) values(DEFAULT)");
+        checkQuery("insert into Person default values");
+        checkQuery("insert into Person() values()");
+        checkQuery("insert into Person(name) values(DEFAULT), (null)");
+        checkQuery("insert into Person(name) values(DEFAULT),");
+        checkQuery("insert into Person(name, parentName) values(DEFAULT, null), (?, ?)");
+        checkQuery("insert into Person(old, name) values(5, 'John',), (6, 'Jack')");
+        checkQuery("insert into Person(old, name) values(5 * 3, DEFAULT,)");
+        checkQuery("insert into Person(old, name) values(ABS(-8), 'Max')");
+        checkQuery("insert into Person(old, name) values(5, 'Jane'), (DEFAULT, DEFAULT), (6, 'Jill')");
+        checkQuery("insert into Person(old, name, parentName) values(8 * 7, DEFAULT, 'Unknown')");
+        checkQuery("insert into Person(old, name, parentName) values" +
+            "(2016 - 1828, CONCAT('Leo', 'Tolstoy'), CONCAT(?, 'Tolstoy'))," +
+            "(?, 'AlexanderPushkin', null)," +
+            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), DEFAULT),");
+        checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
+            "('20160112', 1233, 'Ivan Ivanov', 'Peter Ivanov', 123)");
+        checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
+            "(CURRENT_DATE(), RAND(), ASCII('Hi'), INSERT('Leo Tolstoy', 4, 4, 'Max'), ASCII('HI'))");
+        checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
+            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, DEFAULT, DEFAULT)");
+        checkQuery("insert into Person SET old = 5, name = 'John'");
+        checkQuery("insert into Person SET name = CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), old = " +
+            "select (5, 6)");
+        checkQuery("insert into Person(old, name) select ASCII(parentName), INSERT(parentName, 4, 4, 'Max') from " +
+            "Person where date='20110312'");
+
+        /* Subqueries. */
+        checkQuery("insert into Person(old, name) select old, parentName from Person");
+        checkQuery("insert into Person(old, name) direct sorted select old, parentName from Person");
+        checkQuery("insert into Person(old, name) sorted select old, parentName from Person where old > 5");
+        checkQuery("insert into Person(old, name) select 5, 'John'");
+        checkQuery("insert into Person(old, name) select p1.old, 'Name' from person p1 join person p2 on " +
+            "p2.name = p1.parentName where p2.old > 30");
+        checkQuery("insert into Person(old) select 5 from Person UNION select street from Address limit ? offset ?");
+    }
+
+    /** */
+    public void testParseDelete() throws Exception {
+        checkQuery("delete from Person");
+        checkQuery("delete from Person p where p.old > ?");
+        checkQuery("delete from Person where old in (select (40, 41, 42))");
+        checkQuery("delete top 5 from Person where old in (select (40, 41, 42))");
+        checkQuery("delete top ? from Person where old > 5 and length(name) < ?");
+        checkQuery("delete from Person where name in ('Ivan', 'Peter') limit 20");
+        checkQuery("delete from Person where name in ('Ivan', ?) limit ?");
+    }
+
+    /** */
+    public void testParseUpdate() throws Exception {
+        checkQuery("update Person set name='Peter'");
+        checkQuery("update Person per set name='Peter', old = 5");
+        checkQuery("update Person p set name='Peter' limit 20");
+        checkQuery("update Person p set name='Peter', old = length('zzz') limit 20");
+        checkQuery("update Person p set name=DEFAULT, old = null limit ?");
+        checkQuery("update Person p set name=? where old >= ? and old < ? limit ?");
+        checkQuery("update Person p set name=(select a.Street from Address a where a.id=p.addrId), old = (select 42)" +
+            " where old = sqrt(?)");
+        checkQuery("update Person p set (name, old) = (select 'Peter', 42)");
+        checkQuery("update Person p set (name, old) = (select street, id from Address where id > 5 and id <= ?)");
+    }
+
     /**
      *
      */
@@ -315,7 +420,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
     private void checkQuery(String qry) throws Exception {
         Prepared prepared = parse(qry);
 
-        GridSqlQuery gQry = new GridSqlQueryParser().parse(prepared);
+        GridSqlStatement gQry = new GridSqlQueryParser().parse(prepared);
 
         String res = gQry.getSQL();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 7f98d0a..783f831 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgume
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCrossCacheJoinRandomTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheDeleteSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest;
@@ -37,9 +38,11 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQue
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheInsertSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheJoinQueryWithAffinityKeyTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheMergeSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapIndexScanTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
@@ -54,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreaded
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest;
@@ -148,6 +152,11 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheCrossCacheJoinRandomTest.class);
         suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class);
 
+        suite.addTestSuite(IgniteCacheMergeSqlQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheInsertSqlQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheUpdateSqlQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheDeleteSqlQuerySelfTest.class);
+
         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class);
         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapTest.class);
         suite.addTestSuite(IgniteBinaryObjectQueryArgumentsOffheapLocalTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/benchmark-bin-identity.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-bin-identity.properties b/modules/yardstick/config/benchmark-bin-identity.properties
new file mode 100644
index 0000000..3d754c5
--- /dev/null
+++ b/modules/yardstick/config/benchmark-bin-identity.properties
@@ -0,0 +1,94 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#
+# Contains all multicast benchmarks
+#
+
+now0=`date +'%H%M%S'`
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" \
+-Xloggc:./gc${now0}.log \
+-XX:+PrintGCDetails \
+-verbose:gc \
+-XX:+UseParNewGC \
+-XX:+UseConcMarkSweepGC \
+-XX:+UseTLAB \
+-XX:NewSize=128m \
+-XX:MaxNewSize=128m \
+-XX:MaxTenuringThreshold=0 \
+-XX:SurvivorRatio=1024 \
+-XX:+UseCMSInitiatingOccupancyOnly \
+-XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+#Ignite version
+ver="RELEASE-"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Restart servers for each benchmark.
+RESTART_SERVERS=true
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on.
+SERVER_HOSTS=localhost,localhost,localhost
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on.
+DRIVER_HOSTS=localhost
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Backups count.
+b=1
+
+# Warmup.
+w=60
+
+# Duration.
+d=300
+
+# Threads count.
+t=64
+
+# Sync mode.
+sm=PRIMARY_SYNC
+
+# Jobs.
+j=10
+
+# Run configuration which contains all benchmarks.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteFieldsBinaryIdentityGetBenchmark -sn IgniteNode -ds ${ver}fields-bin-id-atomic-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteFieldsBinaryIdentityPutBenchmark -sn IgniteNode -ds ${ver}fields-bin-id-atomic-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteLegacyBinaryIdentityGetBenchmark -sn IgniteNode -ds ${ver}legacy-bin-id-atomic-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-bin-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteLegacyBinaryIdentityPutBenchmark -sn IgniteNode -ds ${ver}legacy-bin-id-atomic-put-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/benchmark-multicast.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-multicast.properties b/modules/yardstick/config/benchmark-multicast.properties
index 0428c73..3b31745 100644
--- a/modules/yardstick/config/benchmark-multicast.properties
+++ b/modules/yardstick/config/benchmark-multicast.properties
@@ -117,6 +117,21 @@ CONFIGS="\
 -cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -bs 100 -dn IgnitePutAllBenchmark -sn IgniteNode -ds ${ver}atomic-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 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 100 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds ${ver}sql-merge-all-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteReplaceIndexedValue1Benchmark -sn IgniteNode -ds ${ver}replace-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgnitePutIfAbsentIndexedValue1Benchmark -sn IgniteNode -ds ${ver}put-if-absent-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeBenchmark -sn IgniteNode -ds ${ver}sql-merge-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeQueryBenchmark -sn IgniteNode -ds ${ver}sql-merge-query-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeIndexedValue1Benchmark -sn IgniteNode -ds ${ver}sql-merge-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeIndexedValue2Benchmark -sn IgniteNode -ds ${ver}sql-merge-indexed2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlMergeIndexedValue8Benchmark -sn IgniteNode -ds ${ver}sql-merge-indexed8-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlInsertIndexedValue1Benchmark -sn IgniteNode -ds ${ver}sql-insert-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlInsertIndexedValue2Benchmark -sn IgniteNode -ds ${ver}sql-insert-indexed2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlInsertIndexedValue8Benchmark -sn IgniteNode -ds ${ver}sql-insert-indexed8-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -r 300000 -dn IgniteSqlDeleteBenchmark -sn IgniteNode -ds ${ver}sql-delete-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlDeleteFilteredBenchmark -sn IgniteNode -ds ${ver}sql-delete-filtered-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlUpdateBenchmark -sn IgniteNode -ds ${ver}sql-update-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-multicast-config.xml -nn ${nodesNum} -b ${b} -w ${w} -d ${d} -t ${t} -sm ${sm} -dn IgniteSqlUpdateFilteredBenchmark -sn IgniteNode -ds ${ver}sql-update-filtered-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,\

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/benchmark-sql-dml.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-sql-dml.properties b/modules/yardstick/config/benchmark-sql-dml.properties
new file mode 100644
index 0000000..2ce2e1f
--- /dev/null
+++ b/modules/yardstick/config/benchmark-sql-dml.properties
@@ -0,0 +1,72 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#
+# Contains benchmarks for SQL queries.
+#
+
+# JVM options.
+# JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
+  -XX:+UseParNewGC \
+  -XX:+UseConcMarkSweepGC \
+  -XX:+UseTLAB \
+  -XX:NewSize=128m \
+  -XX:MaxNewSize=128m \
+  -XX:MaxTenuringThreshold=0 \
+  -XX:SurvivorRatio=1024 \
+  -XX:+UseCMSInitiatingOccupancyOnly \
+  -XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
+SERVER_HOSTS=localhost,localhost
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+DRIVER_HOSTS=localhost
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Run configuration.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+
+#
+
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 2 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds sql-merge-all-bs-2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 6 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds sql-merge-all-bs-6-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -bs 10 -dn IgniteSqlMergeAllBenchmark -sn IgniteNode -ds sql-merge-all-bs-10-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeBenchmark -sn IgniteNode -ds sql-merge-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeQueryBenchmark -sn IgniteNode -ds sql-merge-query-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeIndexedValue1Benchmark -sn IgniteNode -ds sql-merge-indexed1-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeIndexedValue2Benchmark -sn IgniteNode -ds sql-merge-indexed2-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlMergeIndexedValue8Benchmark -sn IgniteNode -ds sql-merge-indexed8-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/ignite-base-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-base-config.xml b/modules/yardstick/config/ignite-base-config.xml
index 346f507..615cb42 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -127,7 +127,14 @@
                 </bean>
 
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="query"/>
+                    <property name="name" value="atomic-index-with-eviction"/>
+
+                    <property name="evictionPolicy">
+                        <bean class="org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy">
+                            <!-- default range (1000000) x (1 + default backups number (1)) / default nodes number (4) -->
+                            <constructor-arg value="500000"/>
+                        </bean>
+                    </property>
 
                     <property name="cacheMode" value="PARTITIONED"/>
 
@@ -135,38 +142,40 @@
 
                     <property name="swapEnabled" value="false"/>
 
-                    <property name="typeMetadata">
+                    <property name="indexedTypes">
                         <list>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                                <property name="keyType" value="java.lang.Integer"/>
-                                <property name="valueType" value="org.apache.ignite.yardstick.cache.model.Organization"/>
-
-                                <property name="ascendingFields">
-                                    <map>
-                                        <entry key="id" value="java.lang.Integer"/>
-                                        <entry key="name" value="java.lang.String"/>
-                                    </map>
-                                </property>
-                            </bean>
-                            <bean class="org.apache.ignite.cache.CacheTypeMetadata">
-                                <property name="keyType" value="java.lang.Integer"/>
-                                <property name="valueType" value="org.apache.ignite.yardstick.cache.model.Person"/>
-
-                                <property name="ascendingFields">
-                                    <map>
-                                        <entry key="id" value="java.lang.Integer"/>
-                                        <entry key="orgId" value="java.lang.Integer"/>
-                                        <entry key="salary" value="java.lang.Double"/>
-                                    </map>
-                                </property>
-
-                                <property name="queryFields">
-                                    <map>
-                                        <entry key="firstName" value="java.lang.String"/>
-                                        <entry key="lastName" value="java.lang.String"/>
-                                    </map>
-                                </property>
-                            </bean>
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person1</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person2</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person8</value>
+                        </list>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="query"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="ATOMIC"/>
+
+                    <property name="indexedTypes">
+                        <list>
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Organization</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.SampleValue</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>java.lang.Integer</value>
                         </list>
                     </property>
                 </bean>

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/config/ignite-bin-multicast-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-bin-multicast-config.xml b/modules/yardstick/config/ignite-bin-multicast-config.xml
new file mode 100644
index 0000000..e7eb6a1
--- /dev/null
+++ b/modules/yardstick/config/ignite-bin-multicast-config.xml
@@ -0,0 +1,86 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<!--
+    Ignite Spring configuration file to startup grid.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.5.xsd">
+    <import resource="ignite-base-config.xml"/>
+
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-ignite.cfg">
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder"/>
+                </property>
+            </bean>
+        </property>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.binary.BinaryMarshaller"/>
+        </property>
+
+        <property name="cacheKeyConfiguration">
+            <list>
+                <bean class="org.apache.ignite.cache.CacheKeyConfiguration">
+                    <property name="typeName" value="BinaryKeyWithLegacyIdentity"/>
+
+                    <property name="affinityKeyFieldName" value="f3" />
+                </bean>
+
+                <bean class="org.apache.ignite.cache.CacheKeyConfiguration">
+                    <property name="typeName" value="BinaryKeyWithFieldsIdentity"/>
+
+                    <property name="affinityKeyFieldName" value="f3" />
+                </bean>
+            </list>
+        </property>
+
+        <property name="binaryConfiguration">
+            <bean class="org.apache.ignite.configuration.BinaryConfiguration">
+                <property name="typeConfigurations">
+                    <list>
+                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
+                            <property name="typeName" value="BinaryKeyWithLegacyIdentity"/>
+                        </bean>
+
+                        <bean class="org.apache.ignite.binary.BinaryTypeConfiguration">
+                            <property name="typeName" value="BinaryKeyWithFieldsIdentity"/>
+
+                            <property name="identity">
+                                <bean class="org.apache.ignite.binary.BinaryFieldIdentityResolver">
+                                    <property name="fieldNames">
+                                        <list>
+                                            <value>f1</value>
+                                            <value>f2</value>
+                                            <value>f3</value>
+                                        </list>
+                                    </property>
+                                </bean>
+                            </property>
+                        </bean>
+                    </list>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java
new file mode 100644
index 0000000..a2e29dd
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityBenchmark.java
@@ -0,0 +1,108 @@
+/*
+ * 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 org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectBuilder;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs get operations.
+ */
+abstract class IgniteBinaryIdentityBenchmark extends IgniteCacheAbstractBenchmark<BinaryObject, SampleValue> {
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        println(cfg, "Populating data...");
+
+        long start = System.nanoTime();
+
+        try (IgniteDataStreamer<BinaryObject, Object> dataLdr = ignite().dataStreamer(cache.getName())) {
+            for (int i = 0; i < args.range() && !Thread.currentThread().isInterrupted();) {
+                dataLdr.addData(createKey(i), new SampleValue(i));
+
+                if (++i % 100000 == 0)
+                    println(cfg, "Items populated: " + i);
+            }
+        }
+
+        println(cfg, "Finished populating data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<BinaryObject, SampleValue> cache() {
+        return ignite().cache("atomic").withKeepBinary();
+    }
+
+    /**
+     * @param key Base key value.
+     * @return Binary key.
+     */
+    abstract BinaryObject createKey(int key);
+
+    /**
+     * @param key Key field value.
+     * @return Binary object without hash code explicitly set at build time.
+     */
+    BinaryObject createFieldsIdentityBinaryKey(int key) {
+        BinaryObjectBuilder bldr = ignite().binary().builder("BinaryKeyWithFieldsIdentity");
+
+        setBuilderFields(bldr, key);
+
+        return bldr.build();
+    }
+
+    /**
+     * @param key Key field value.
+     * @return Binary object with hash code explicitly set at build time.
+     */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
+    BinaryObject createLegacyIdentityBinaryKey(int key) {
+        BinaryObjectBuilder bldr = ignite().binary().builder("BinaryKeyWithLegacyIdentity");
+
+        setBuilderFields(bldr, key);
+
+        int hash = 0;
+
+        hash = 31 * hash + 1;
+        hash = 31 * hash + ("SomeString".hashCode());
+        hash = 31 * hash + Long.valueOf(key).hashCode();
+
+        bldr.hashCode(hash);
+
+        return bldr.build();
+    }
+
+    /**
+     * @param builder Builder.
+     * @param key Key field value.
+     */
+    private static void setBuilderFields(BinaryObjectBuilder builder, int key) {
+        builder.setField("f1", 1);
+
+        builder.setField("f2", "SomeString");
+
+        builder.setField("f3", (long) key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java
new file mode 100644
index 0000000..1af1846
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityGetBenchmark.java
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+/**
+ * Test GETs with binary hashed key.
+ */
+public abstract class IgniteBinaryIdentityGetBenchmark extends IgniteBinaryIdentityBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.get(createKey(key));
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java
new file mode 100644
index 0000000..58895a0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteBinaryIdentityPutBenchmark.java
@@ -0,0 +1,35 @@
+/*
+ * 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.yardstick.cache.model.SampleValue;
+
+/**
+ * Test PUTs with binary hashed key.
+ */
+public abstract class IgniteBinaryIdentityPutBenchmark extends IgniteBinaryIdentityBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(createKey(key), new SampleValue(key));
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java
new file mode 100644
index 0000000..333b245
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityGetBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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 org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test GETs with binary hashed key.
+ */
+public class IgniteFieldsBinaryIdentityGetBenchmark extends IgniteBinaryIdentityGetBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createFieldsIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java
new file mode 100644
index 0000000..c905845
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteFieldsBinaryIdentityPutBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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 org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test PUTs with binary hashed key.
+ */
+public class IgniteFieldsBinaryIdentityPutBenchmark extends IgniteBinaryIdentityPutBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createFieldsIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java
new file mode 100644
index 0000000..1b36b24
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityGetBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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 org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test GETs with binary hashed key.
+ */
+public class IgniteLegacyBinaryIdentityGetBenchmark extends IgniteBinaryIdentityGetBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createLegacyIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java
new file mode 100644
index 0000000..246fc28
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteLegacyBinaryIdentityPutBenchmark.java
@@ -0,0 +1,30 @@
+/*
+ * 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 org.apache.ignite.binary.BinaryObject;
+
+/**
+ * Test PUTs with binary hashed key.
+ */
+public class IgniteLegacyBinaryIdentityPutBenchmark extends IgniteBinaryIdentityPutBenchmark {
+    /** {@inheritDoc} */
+    @Override BinaryObject createKey(int key) {
+        return createLegacyIdentityBinaryKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java
new file mode 100644
index 0000000..aea909a
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIfAbsentIndexedValue1Benchmark.java
@@ -0,0 +1,45 @@
+/*
+ * 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.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.cache.model.Person1;
+
+/**
+ * Ignite benchmark that performs putIfAbsent operations for entity with indexed fields.
+ */
+public class IgnitePutIfAbsentIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.getAndIncrement();
+
+        cache.putIfAbsent(key, new Person1(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java
new file mode 100644
index 0000000..cc50c84
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteReplaceIndexedValue1Benchmark.java
@@ -0,0 +1,79 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.yardstick.cache.model.Person1;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ * Ignite benchmark that performs SQL UPDATE operations.
+ */
+public class IgniteReplaceIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public void setUp(final BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        final AtomicInteger i = new AtomicInteger();
+
+        Collection<Thread> setupThreads = new ArrayList<>(cfg.threads());
+
+        for (int j = 0; j < cfg.threads(); j++) {
+            Thread t = new Thread() {
+                /** {@inheritDoc} */
+                @Override public void run() {
+                    int k;
+
+                    while ((k = i.getAndIncrement()) < args.range()) {
+                        cache().put(k, new Person1(k));
+                        if (++k % 100000 == 0)
+                            BenchmarkUtils.println(cfg, "REPLACE setUp: have successfully put " + k + " items");
+                    }
+                }
+            };
+
+            setupThreads.add(t);
+
+            t.start();
+        }
+
+        for (Thread t : setupThreads)
+            t.join();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        cache.replace(rnd.nextInt(args.range()), new Person1(rnd.nextInt(args.range())));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java
new file mode 100644
index 0000000..864211c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteBenchmark.java
@@ -0,0 +1,83 @@
+/*
+ * 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.dml;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person1;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ * Ignite benchmark that performs SQL DELETE operations.
+ */
+public class IgniteSqlDeleteBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final ConcurrentLinkedQueue<Integer> keys = new ConcurrentLinkedQueue<>();
+
+    /** {@inheritDoc} */
+    @Override public void setUp(final BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        final AtomicInteger i = new AtomicInteger();
+
+        Collection<Thread> setupThreads = new ArrayList<>(cfg.threads());
+
+        for (int j = 0; j < cfg.threads(); j++) {
+            Thread t = new Thread() {
+                /** {@inheritDoc} */
+                @Override public void run() {
+                    int k;
+
+                    while ((k = i.getAndIncrement()) < args.range()) {
+                        cache().put(k, new Person1(k));
+                        keys.add(k);
+                        if (k % 100000 == 0)
+                            BenchmarkUtils.println(cfg, "DELETE setUp: have successfully put " + k + " items");
+                    }
+                }
+            };
+
+            setupThreads.add(t);
+
+            t.start();
+        }
+
+        for (Thread t : setupThreads)
+            t.join();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        cache.query(new SqlFieldsQuery("delete from Person1 where _key = ?").setArgs(keys.remove()));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java
new file mode 100644
index 0000000..218e8dc
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlDeleteFilteredBenchmark.java
@@ -0,0 +1,88 @@
+/*
+ * 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.dml;
+
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person;
+import org.yardstickframework.BenchmarkConfiguration;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ * Ignite benchmark that performs put and SQL DELETE operations.
+ */
+public class IgniteSqlDeleteFilteredBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private AtomicInteger putCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger delCnt = new AtomicInteger();
+
+    /** */
+    private AtomicInteger delItemsCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        if (rnd.nextBoolean()) {
+            double salary = rnd.nextDouble() * args.range() * 1000;
+
+            double maxSalary = salary + 1000;
+
+            int res = (Integer) cache().query(new SqlFieldsQuery("delete from Person where salary >= ? and salary <= ?")
+                .setArgs(salary, maxSalary)).getAll().get(0).get(0);
+
+            delItemsCnt.getAndAdd(res);
+
+            delCnt.getAndIncrement();
+        }
+        else {
+            int i = rnd.nextInt(args.range());
+
+            cache.put(i, new Person(i, "firstName" + i, "lastName" + i, i * 1000));
+
+            putCnt.getAndIncrement();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        println(cfg, "Finished SQL DELETE query benchmark [putCnt=" + putCnt.get() + ", delCnt=" + delCnt.get() +
+            ", delItemsCnt=" + delItemsCnt.get() + ']');
+
+        super.tearDown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java
new file mode 100644
index 0000000..63a585d
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue1Benchmark.java
@@ -0,0 +1,48 @@
+/*
+ * 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.dml;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person1;
+
+/**
+ * Ignite benchmark that performs SQL INSERT operations for entity with 1 indexed field.
+ */
+public class IgniteSqlInsertIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.getAndIncrement();
+
+        cache.query(new SqlFieldsQuery("insert into Person1(_key, _val) values (?, ?)")
+            .setArgs(key, new Person1(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java
new file mode 100644
index 0000000..b099951
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue2Benchmark.java
@@ -0,0 +1,48 @@
+/*
+ * 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.dml;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person2;
+
+/**
+ * Ignite benchmark that performs SQL INSERT operations for entity with 2 indexed fields.
+ */
+public class IgniteSqlInsertIndexedValue2Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.getAndIncrement();
+
+        cache.query(new SqlFieldsQuery("insert into Person2(_key, _val) values (?, ?)")
+            .setArgs(key, new Person2(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java
new file mode 100644
index 0000000..3193f38
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlInsertIndexedValue8Benchmark.java
@@ -0,0 +1,48 @@
+/*
+ * 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.dml;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.Person8;
+
+/**
+ * Ignite benchmark that performs SQL INSERT operations for entity with 8 indexed fields.
+ */
+public class IgniteSqlInsertIndexedValue8Benchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private final AtomicInteger insCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = insCnt.incrementAndGet();
+
+        cache.query(new SqlFieldsQuery("insert into Person8(_key, _val) values (?, ?)")
+            .setArgs(key, new Person8(key)));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index-with-eviction");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java
new file mode 100644
index 0000000..dc8c105
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeAllBenchmark.java
@@ -0,0 +1,82 @@
+/*
+ * 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.dml;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+
+/**
+ * Ignite benchmark that performs putAll operations via SQL MERGE.
+ */
+public class IgniteSqlMergeAllBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** Affinity mapper. */
+    private Affinity<Integer> aff;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        aff = ignite().affinity("query");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        Object[] vals = new Object[args.batch() * 2];
+
+        ClusterNode node = args.collocated() ? aff.mapKeyToNode(nextRandom(args.range())) : null;
+
+        StringBuilder qry = new StringBuilder("merge into Integer(_key, _val) values ");
+
+        int j = 0;
+
+        for (int i = 0; i < args.batch(); ) {
+            int key = nextRandom(args.range());
+
+            if (args.collocated() && !aff.isPrimary(node, key))
+                continue;
+
+            ++i;
+
+            // Put two args, for key and value.
+            vals[j++] = key;
+            vals[j++] = key;
+
+            qry.append("(?, ?),");
+        }
+
+        Arrays.sort(vals);
+
+        // Trim trailing comma.
+        qry = qry.deleteCharAt(qry.length() - 1);
+
+        cache.query(new SqlFieldsQuery(qry.toString()).setArgs(vals));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/86d143bb/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java
new file mode 100644
index 0000000..3b93403
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/dml/IgniteSqlMergeBenchmark.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dml;
+
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.yardstick.cache.IgniteCacheAbstractBenchmark;
+
+/**
+ * Ignite benchmark that performs SQL MERGE operations.
+ */
+public class IgniteSqlMergeBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.query(new SqlFieldsQuery("merge into SampleValue(_key, id) values(?, ?)").setArgs(key, key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("query");
+    }
+}

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

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

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