You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/02/03 07:32:18 UTC

[17/46] incubator-ignite git commit: # IGNITE-102: Initial benchmark code.

# IGNITE-102: Initial benchmark code.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/b0031ccd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/b0031ccd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/b0031ccd

Branch: refs/heads/ingite-9655-merge
Commit: b0031ccda7379a80047ae429ccb7da7be2a0139e
Parents: 9660a1c
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jan 27 16:16:31 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jan 27 16:16:31 2015 +0700

----------------------------------------------------------------------
 modules/yardstick/README.md                     |  67 ++++++
 modules/yardstick/pom.xml                       | 177 +++++++++++++++
 .../yardstick/IgniteAbstractBenchmark.java      | 130 +++++++++++
 .../yardstick/IgniteBenchmarkArguments.java     | 213 +++++++++++++++++++
 .../org/apache/ignite/yardstick/IgniteNode.java | 191 +++++++++++++++++
 .../yardstick/cache/GetBenchmarkIgnite.java     |  41 ++++
 .../cache/IgniteCacheAbstractBenchmark.java     |  44 ++++
 .../cache/IgnitePutBenchmarkIgnite.java         |  42 ++++
 .../cache/IgnitePutGetBenchmarkIgnite.java      |  47 ++++
 .../cache/IgnitePutGetTxBenchmarkIgnite.java    |  52 +++++
 .../cache/IgnitePutTxBenchmarkIgnite.java       |  43 ++++
 .../cache/IgniteSqlQueryBenchmarkIgnite.java    |  94 ++++++++
 .../IgniteSqlQueryJoinBenchmarkIgnite.java      | 118 ++++++++++
 .../cache/IgniteSqlQueryPutBenchmarkIgnite.java |  86 ++++++++
 .../yardstick/cache/model/Organization.java     | 110 ++++++++++
 .../ignite/yardstick/cache/model/Person.java    | 191 +++++++++++++++++
 .../yardstick/cache/model/SampleValue.java      |  62 ++++++
 .../compute/IgniteAffinityCallBenchmark.java    |  36 ++++
 .../yardstick/compute/IgniteApplyBenchmark.java |  72 +++++++
 .../compute/IgniteBroadcastBenchmark.java       |  35 +++
 .../compute/IgniteExecuteBenchmark.java         |  35 +++
 .../yardstick/compute/IgniteRunBenchmark.java   |  71 +++++++
 .../yardstick/compute/model/NoopCallable.java   |  41 ++++
 .../yardstick/compute/model/NoopTask.java       |  97 +++++++++
 pom.xml                                         |  13 +-
 25 files changed, 2107 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/README.md
----------------------------------------------------------------------
diff --git a/modules/yardstick/README.md b/modules/yardstick/README.md
new file mode 100644
index 0000000..d2aebc0
--- /dev/null
+++ b/modules/yardstick/README.md
@@ -0,0 +1,67 @@
+# Yardstick Ignite Benchmarks
+Yardstick Ignite is a set of <a href="http://ignite.incubator.apache.org/">Ignite Grid</a> benchmarks written on top of Yardstick framework.
+
+## Yardstick Framework
+Visit <a href="https://github.com/gridgain/yardstick" target="_blank">Yardstick Repository</a> for detailed information on how to run Yardstick benchmarks and how to generate graphs.
+
+The documentation below describes configuration parameters in addition to standard Yardstick parameters.
+
+## Installation
+1. Create a local clone of Ignite repository
+2. Run `mvn package` command for Yardstick Ignite POM
+
+## Provided Benchmarks
+The following benchmarks are provided:
+
+1. `GetBenchmark` - benchmarks atomic distributed cache get operation
+2. `PutBenchmark` - benchmarks atomic distributed cache put operation
+3. `PutGetBenchmark` - benchmarks atomic distributed cache put and get operations together
+4. `PutTxBenchmark` - benchmarks transactional distributed cache put operation
+5. `PutGetTxBenchmark` - benchmarks transactional distributed cache put and get operations together
+6. `SqlQueryBenchmark` - benchmarks distributed SQL query over cached data
+7. `SqlQueryJoinBenchmark` - benchmarks distributed SQL query with a Join over cached data
+8. `SqlQueryPutBenchmark` - benchmarks distributed SQL query with simultaneous cache updates
+
+## Writing Ignite Benchmarks
+All benchmarks extend `AbstractBenchmark` class. A new benchmark should also extend this abstract class and implement `test` method. This is the method that is actually benchmarked.
+
+## Running Ignite Benchmarks
+Before running Ignite benchmarks, run `mvn package` command. This command will compile the project and also will unpack scripts from `yardstick-resources.zip` file to `bin` directory.
+
+### Properties And Command Line Arguments
+> Note that this section only describes configuration parameters specific to Ignite benchmarks, and not for Yardstick framework. To run Ignite benchmarks and generate graphs, you will need to run them using Yardstick framework scripts in `bin` folder.
+
+> Refer to [Yardstick Documentation](https://github.com/gridgain/yardstick) for common Yardstick properties and command line arguments for running Yardstick scripts.
+
+The following Ignite benchmark properties can be defined in the benchmark configuration:
+
+* `-nn <num>` or `--nodeNumber <num>` - Number of nodes (automatically set in `benchmark.properties`), used to wait for the specified number of nodes to start
+* `-b <num>` or `--backups <num>` - Number of backups for every key
+* `-ggcfg <path>` or `--ggConfig <path>` - Path to Ignite configuration file
+* `-sm <mode>` or `-syncMode <mode>` - Synchronization mode (defined in `CacheWriteSynchronizationMode`)
+* `-dm <mode>` or `--distroMode <mode>` - Distribution mode (defined in `CacheDistributionMode`)
+* `-wom <mode>` or `--writeOrderMode <mode>` - Write order mode for ATOMIC caches (defined in `CacheAtomicWriteOrderMode`)
+* `-txc <value>` or `--txConcurrency <value>` - Cache transaction concurrency control, either `OPTIMISTIC` or `PESSIMISTIC` (defined in `CacheTxConcurrency`)
+* `-txi <value>` or `--txIsolation <value>` - Cache transaction isolation (defined in `CacheTxIsolation`)
+* `-ot` or `--offheapTiered` - Flag indicating whether tiered off-heap mode is on
+* `-ov` or `--offheapValuesOnly` - Flag indicating whether off-heap mode is on and only cache values are stored off-heap
+* `-rtp <num>`  or `--restPort <num>` - REST TCP port, indicates that a Ignite node is ready to process Ignite Clients
+* `-rth <host>` or `--restHost <host>` - REST TCP host
+* `-ss` or `--syncSend` - Flag indicating whether synchronous send is used in `TcpCommunicationSpi`
+* `-r <num>` or `--range` - Range of keys that are randomly generated for cache operations
+
+For example if we need to run 2 `IgniteNode` servers on localhost with `PutBenchmark` benchmark on localhost, with number of backups set to 1, synchronization mode set to `PRIMARY_SYNC`, then the following configuration should be specified in `benchmark.properties` file:
+
+```
+SERVER_HOSTS=localhost,localhost
+    
+# Note that -dn and -sn, which stand for data node and server node, are 
+# native Yardstick parameters and are documented in Yardstick framework.
+CONFIGS="-b 1 -sm PRIMARY_SYNC -dn PutBenchmark -sn IgniteNode"
+```
+
+## Issues
+Use Ignite Apache JIRA (https://issues.apache.org/jira/browse/IGNITE) to file bugs.
+
+## License
+Yardstick Ignite is available under [Apache 2.0](http://www.apache.org/licenses/LICENSE-2.0.html) Open Source license.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
new file mode 100644
index 0000000..67052dc
--- /dev/null
+++ b/modules/yardstick/pom.xml
@@ -0,0 +1,177 @@
+<?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.
+-->
+
+<!--
+    POM file.
+-->
+<project
+    xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite</artifactId>
+        <version>${ignite.version}</version>
+        <relativePath>../..</relativePath>
+    </parent>
+
+    <artifactId>ignite-yardstick</artifactId>
+
+    <properties>
+        <yardstick.version>0.7.0</yardstick.version>
+    </properties>
+
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+            <version>${ignite.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${ignite.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
+            <version>${ignite.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.yardstickframework</groupId>
+            <artifactId>yardstick</artifactId>
+            <version>${yardstick.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.beust</groupId>
+            <artifactId>jcommander</artifactId>
+            <version>1.32</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-core</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-beans</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-context</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-expression</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <version>3.1</version>
+                <configuration >
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>2.8</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${basedir}/libs</outputDirectory>
+                            <excludeTypes>pom</excludeTypes>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>unpack</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>unpack</goal>
+                        </goals>
+                        <configuration>
+                            <artifactItems>
+                                <artifactItem>
+                                    <groupId>org.yardstickframework</groupId>
+                                    <artifactId>yardstick</artifactId>
+                                    <version>${yardstick.version}</version>
+                                    <type>zip</type>
+                                    <classifier>resources</classifier>
+                                    <outputDirectory>${basedir}</outputDirectory>
+                                </artifactItem>
+                            </artifactItems>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <version>2.4</version>
+                <configuration>
+                    <outputDirectory>${basedir}/libs</outputDirectory>
+                </configuration>
+            </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-clean-plugin</artifactId>
+                <version>2.5</version>
+                <configuration>
+                    <filesets>
+                        <fileset>
+                            <directory>${basedir}/bin</directory>
+                        </fileset>
+                        <fileset>
+                            <directory>${basedir}/libs</directory>
+                            <includes>
+                                <include>**/*.jar</include>
+                            </includes>
+                        </fileset>
+                    </filesets>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
new file mode 100644
index 0000000..5cbfee1
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
@@ -0,0 +1,130 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+import org.yardstickframework.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.events.IgniteEventType.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+import static org.yardstickframework.BenchmarkUtils.*;
+
+/**
+ * Abstract class for GridGain benchmarks.
+ */
+public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
+    /** Arguments. */
+    protected final IgniteBenchmarkArguments args = new IgniteBenchmarkArguments();
+
+    /** Node. */
+    private IgniteNode node;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        jcommander(cfg.commandLineArguments(), args, "<gridgain-driver>");
+
+        if (Ignition.state() != IgniteState.STARTED) {
+            node = new IgniteNode(args.distributionMode() == CLIENT_ONLY);
+
+            node.start(cfg);
+        }
+        else
+            // Support for mixed benchmarks mode.
+            node = new IgniteNode(args.distributionMode() == CLIENT_ONLY, Ignition.ignite());
+
+        waitForNodes();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        if (node != null)
+            node.stop();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String description() {
+        String desc = BenchmarkUtils.description(cfg, this);
+
+        return desc.isEmpty() ?
+            getClass().getSimpleName() + args.description() + cfg.defaultDescription() : desc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String usage() {
+        return BenchmarkUtils.usage(args);
+    }
+
+    /**
+     * @return Grid.
+     */
+    protected Ignite grid() {
+        return node.grid();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void waitForNodes() throws Exception {
+        final CountDownLatch nodesStartedLatch = new CountDownLatch(1);
+
+        grid().events().localListen(new IgnitePredicate<IgniteEvent>() {
+            @Override public boolean apply(IgniteEvent gridEvt) {
+                if (nodesStarted())
+                    nodesStartedLatch.countDown();
+
+                return true;
+            }
+        }, EVT_NODE_JOINED);
+
+        if (!nodesStarted()) {
+            println(cfg, "Waiting for " + (args.nodes() - 1) + " nodes to start...");
+
+            nodesStartedLatch.await();
+        }
+    }
+
+    /**
+     * @return {@code True} if all nodes are started, {@code false} otherwise.
+     */
+    private boolean nodesStarted() {
+        return grid().cluster().nodes().size() >= args.nodes();
+    }
+
+    /**
+     * @param max Key range.
+     * @return Next key.
+     */
+    protected int nextRandom(int max) {
+        return ThreadLocalRandom.current().nextInt(max);
+    }
+
+    /**
+     * @param min Minimum key in range.
+     * @param max Maximum key in range.
+     * @return Next key.
+     */
+    protected int nextRandom(int min, int max) {
+        return ThreadLocalRandom.current().nextInt(max - min) + min;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
new file mode 100644
index 0000000..ebcddf3
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -0,0 +1,213 @@
+/*
+ * 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;
+
+import com.beust.jcommander.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.transactions.*;
+
+/**
+ * Input arguments for GridGain benchmarks.
+ */
+@SuppressWarnings({"UnusedDeclaration", "FieldCanBeLocal"})
+public class IgniteBenchmarkArguments {
+    /** */
+    @Parameter(names = {"-nn", "--nodeNumber"}, description = "Node number")
+    private int nodes = 1;
+
+    /** */
+    @Parameter(names = {"-b", "--backups"}, description = "Backups")
+    private int backups;
+
+    @Parameter(names = {"-ggcfg", "--ggConfig"}, description = "Configuration file")
+    private String ggcfg = "config/gridgain-localhost-config.xml";
+
+    /** */
+    @Parameter(names = {"-sm", "--syncMode"}, description = "Synchronization mode")
+    private CacheWriteSynchronizationMode syncMode = CacheWriteSynchronizationMode.PRIMARY_SYNC;
+
+    /** */
+    @Parameter(names = {"-dm", "--distroMode"}, description = "Distribution mode")
+    private CacheDistributionMode distroMode = CacheDistributionMode.PARTITIONED_ONLY;
+
+    /** */
+    @Parameter(names = {"-wom", "--writeOrderMode"}, description = "Write ordering mode")
+    private CacheAtomicWriteOrderMode orderMode;
+
+    /** */
+    @Parameter(names = {"-txc", "--txConcurrency"}, description = "Transaction concurrency")
+    private IgniteTxConcurrency txConcurrency = IgniteTxConcurrency.OPTIMISTIC;
+
+    /** */
+    @Parameter(names = {"-txi", "--txIsolation"}, description = "Transaction isolation")
+    private IgniteTxIsolation txIsolation = IgniteTxIsolation.REPEATABLE_READ;
+
+    /** */
+    @Parameter(names = {"-ot", "--offheapTiered"}, description = "Tiered offheap")
+    private boolean offheapTiered;
+
+    /** */
+    @Parameter(names = {"-ov", "--offheapValuesOnly"}, description = "Offheap values only")
+    private boolean offheapVals;
+
+    /** */
+    @Parameter(names = {"-rtp", "--restPort"}, description = "REST TCP port")
+    private int restTcpPort;
+
+    /** */
+    @Parameter(names = {"-rth", "--restHost"}, description = "REST TCP host")
+    private String restTcpHost;
+
+    /** */
+    @Parameter(names = {"-ss", "--syncSend"}, description = "Synchronous send")
+    private boolean syncSnd;
+
+    /** */
+    @Parameter(names = {"-r", "--range"}, description = "Key range")
+    private int range = 1_000_000;
+
+    /** */
+    @Parameter(names = {"-j", "--jobs"}, description = "Number of jobs for compute benchmarks")
+    private int jobs = 10;
+
+    /**
+     * @return Transaction concurrency.
+     */
+    public IgniteTxConcurrency txConcurrency() {
+        return txConcurrency;
+    }
+
+    /**
+     * @return Transaction isolation.
+     */
+    public IgniteTxIsolation txIsolation() {
+        return txIsolation;
+    }
+
+    /**
+     * @return REST TCP port.
+     */
+    public int restTcpPort() {
+        return restTcpPort;
+    }
+
+    /**
+     * @return REST TCP host.
+     */
+    public String restTcpHost() {
+        return restTcpHost;
+    }
+
+    /**
+     * @return Distribution.
+     */
+    public CacheDistributionMode distributionMode() {
+        return distroMode;
+    }
+
+    /**
+     * @return Synchronization.
+     */
+    public CacheWriteSynchronizationMode syncMode() {
+        return syncMode;
+    }
+
+    /**
+     * @return Cache write ordering mode.
+     */
+    public CacheAtomicWriteOrderMode orderMode() {
+        return orderMode;
+    }
+
+    /**
+     * @return Backups.
+     */
+    public int backups() {
+        return backups;
+    }
+
+    /**
+     * @return Offheap tiered.
+     */
+    public boolean isOffheapTiered() {
+        return offheapTiered;
+    }
+
+    /**
+     * @return Offheap values.
+     */
+    public boolean isOffheapValues() {
+        return offheapVals;
+    }
+
+    /**
+     * @return {@code True} if any offheap is enabled.
+     */
+    public boolean isOffHeap() {
+        return offheapTiered || offheapVals;
+    }
+
+    /**
+     * @return Nodes.
+     */
+    public int nodes() {
+        return nodes;
+    }
+
+    /**
+     * @return {@code True} if sending is synchronous.
+     */
+    public boolean isSyncSend() {
+        return syncSnd;
+    }
+
+    /**
+     * @return Key range, from {@code 0} to this number.
+     */
+    public int range() {
+        return range;
+    }
+
+    /**
+     * @return Configuration file.
+     */
+    public String configuration() {
+        return ggcfg;
+    }
+
+    /**
+     * @return Number of jobs
+     */
+    public int jobs() {
+        return jobs;
+    }
+
+    /**
+     * @return Description.
+     */
+    public String description() {
+        return "-nn=" + nodes + "-b=" + backups + "-sm=" + syncMode + "-dm=" + distroMode +
+            (orderMode == null ? "" : "-wom=" + orderMode) + "-txc=" + txConcurrency;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return GridToStringBuilder.toString(IgniteBenchmarkArguments.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
new file mode 100644
index 0000000..38a61be
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -0,0 +1,191 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.springframework.beans.*;
+import org.springframework.beans.factory.xml.*;
+import org.springframework.context.support.*;
+import org.springframework.core.io.*;
+import org.yardstickframework.*;
+
+import java.net.*;
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+
+/**
+ * Standalone GridGain node.
+ */
+public class IgniteNode implements BenchmarkServer {
+    /** Grid instance. */
+    private Ignite grid;
+
+    /** Client mode. */
+    private boolean clientMode;
+
+    /** */
+    public IgniteNode() {
+        // No-op.
+    }
+
+    /** */
+    public IgniteNode(boolean clientMode) {
+        this.clientMode = clientMode;
+    }
+
+    /** */
+    public IgniteNode(boolean clientMode, Ignite grid) {
+        this.clientMode = clientMode;
+        this.grid = grid;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start(BenchmarkConfiguration cfg) throws Exception {
+        IgniteBenchmarkArguments args = new IgniteBenchmarkArguments();
+
+        BenchmarkUtils.jcommander(cfg.commandLineArguments(), args, "<gridgain-node>");
+
+        IgniteConfiguration c = loadConfiguration(args.configuration());
+
+        assert c != null;
+
+        for (CacheConfiguration cc : c.getCacheConfiguration()) {
+            // IgniteNode can not run in CLIENT_ONLY mode,
+            // except the case when it's used inside GridGainAbstractBenchmark.
+            CacheDistributionMode distroMode = args.distributionMode() == CLIENT_ONLY && !clientMode ?
+                PARTITIONED_ONLY : args.distributionMode();
+
+            cc.setWriteSynchronizationMode(args.syncMode());
+            cc.setDistributionMode(distroMode);
+
+            if (args.orderMode() != null)
+                cc.setAtomicWriteOrderMode(args.orderMode());
+
+            cc.setBackups(args.backups());
+
+            if (args.restTcpPort() != 0) {
+                ClientConnectionConfiguration ccc = new ClientConnectionConfiguration();
+
+                ccc.setRestTcpPort(args.restTcpPort());
+
+                if (args.restTcpHost() != null)
+                    ccc.setRestTcpHost(args.restTcpHost());
+
+                c.setClientConnectionConfiguration(ccc);
+            }
+
+            if (args.isOffHeap()) {
+                cc.setOffHeapMaxMemory(0);
+
+                if (args.isOffheapValues())
+                    cc.setMemoryMode(OFFHEAP_VALUES);
+                else
+                    cc.setEvictionPolicy(new CacheLruEvictionPolicy(50000));
+            }
+
+        }
+
+        TransactionsConfiguration tc = c.getTransactionsConfiguration();
+
+        tc.setDefaultTxConcurrency(args.txConcurrency());
+        tc.setDefaultTxIsolation(args.txIsolation());
+
+        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)c.getCommunicationSpi();
+
+        if (commSpi == null)
+            commSpi = new TcpCommunicationSpi();
+
+        c.setCommunicationSpi(commSpi);
+
+        grid = Ignition.start(c);
+    }
+
+    /**
+     * @param springCfgPath Spring configuration file path.
+     * @return Grid configuration.
+     * @throws Exception If failed.
+     */
+    private static IgniteConfiguration loadConfiguration(String springCfgPath) throws Exception {
+        URL url;
+
+        try {
+            url = new URL(springCfgPath);
+        }
+        catch (MalformedURLException e) {
+            url = GridUtils.resolveGridGainUrl(springCfgPath);
+
+            if (url == null)
+                throw new IgniteCheckedException("Spring XML configuration path is invalid: " + springCfgPath +
+                    ". Note that this path should be either absolute or a relative local file system path, " +
+                    "relative to META-INF in classpath or valid URL to GRIDGAIN_HOME.", e);
+        }
+
+        GenericApplicationContext springCtx;
+
+        try {
+            springCtx = new GenericApplicationContext();
+
+            new XmlBeanDefinitionReader(springCtx).loadBeanDefinitions(new UrlResource(url));
+
+            springCtx.refresh();
+        }
+        catch (BeansException e) {
+            throw new Exception("Failed to instantiate Spring XML application context [springUrl=" +
+                url + ", err=" + e.getMessage() + ']', e);
+        }
+
+        Map<String, IgniteConfiguration> cfgMap;
+
+        try {
+            cfgMap = springCtx.getBeansOfType(IgniteConfiguration.class);
+        }
+        catch (BeansException e) {
+            throw new Exception("Failed to instantiate bean [type=" + IgniteConfiguration.class + ", err=" +
+                e.getMessage() + ']', e);
+        }
+
+        if (cfgMap == null || cfgMap.isEmpty())
+            throw new Exception("Failed to find grid configuration in: " + url);
+
+        return cfgMap.values().iterator().next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws Exception {
+        Ignition.stopAll(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String usage() {
+        return BenchmarkUtils.usage(new IgniteBenchmarkArguments());
+    }
+
+    /**
+     * @return Grid.
+     */
+    public Ignite grid() {
+        return grid;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/GetBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/GetBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/GetBenchmarkIgnite.java
new file mode 100644
index 0000000..bbbf4d6
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/GetBenchmarkIgnite.java
@@ -0,0 +1,41 @@
+/*
+ * 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.*;
+
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs get operations.
+ */
+public class GetBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.get(key);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("atomic");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
new file mode 100644
index 0000000..636057c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
@@ -0,0 +1,44 @@
+/*
+ * 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.*;
+import org.apache.ignite.yardstick.*;
+import org.yardstickframework.*;
+
+/**
+ * Abstract class for GridGain benchmarks which use cache.
+ */
+public abstract class IgniteCacheAbstractBenchmark extends IgniteAbstractBenchmark {
+    /** Cache. */
+    protected IgniteCache<Integer, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        cache = cache();
+    }
+
+    /**
+     * Each benchmark must determine which cache will be used.
+     *
+     * @return GridCache Cache to use.
+     */
+    protected abstract IgniteCache<Integer, Object> cache();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmarkIgnite.java
new file mode 100644
index 0000000..fbe55f3
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmarkIgnite.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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.yardstick.cache.model.*;
+
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs put operations.
+ */
+public class IgnitePutBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new SampleValue(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("atomic");
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmarkIgnite.java
new file mode 100644
index 0000000..3cad7d7
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmarkIgnite.java
@@ -0,0 +1,52 @@
+/*
+ * 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.*;
+import org.apache.ignite.transactions.*;
+import org.apache.ignite.yardstick.cache.model.*;
+
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs transactional put and get operations.
+ */
+public class IgnitePutGetTxBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(0, args.range() / 2);
+
+        try (IgniteTx tx = grid().transactions().txStart()) {
+            Object val = cache.get(key);
+
+            if (val != null)
+                key = nextRandom(args.range() / 2, args.range());
+
+            cache.put(key, new SampleValue(key));
+
+            tx.commit();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmarkIgnite.java
new file mode 100644
index 0000000..dd2fd58
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmarkIgnite.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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.yardstick.cache.model.*;
+
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs transactional put operations.
+ */
+public class IgnitePutTxBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        // Implicit transaction is used.
+        cache.put(key, new SampleValue(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("tx");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmarkIgnite.java
new file mode 100644
index 0000000..c769e53
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmarkIgnite.java
@@ -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.
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.yardstick.cache.model.*;
+import org.yardstickframework.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.yardstickframework.BenchmarkUtils.*;
+
+/**
+ * GridGain benchmark that performs query operations.
+ */
+public class IgniteSqlQueryBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** */
+    private CacheQuery qry;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        println(cfg, "Populating query data...");
+
+        long start = System.nanoTime();
+
+        try (IgniteDataLoader<Integer, Person> dataLdr = grid().dataLoader(cache.getName())) {
+            for (int i = 0; i < args.range() && !Thread.currentThread().isInterrupted(); i++) {
+                dataLdr.addData(i, new Person(i, "firstName" + i, "lastName" + i, i * 1000));
+
+                if (i % 100000 == 0)
+                    println(cfg, "Populated persons: " + i);
+            }
+        }
+
+        println(cfg, "Finished populating query data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+
+        qry = null; // TODO: should be fixed after IGNITE-2 cache.queries().createSqlQuery(Person.class, "salary >= ? and salary <= ?");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        double salary = ThreadLocalRandom.current().nextDouble() * args.range() * 1000;
+
+        double maxSalary = salary + 1000;
+
+        Collection<Map.Entry<Integer, Person>> entries = executeQuery(salary, maxSalary);
+
+        for (Map.Entry<Integer, Person> entry : entries) {
+            Person p = entry.getValue();
+
+            if (p.getSalary() < salary || p.getSalary() > maxSalary)
+                throw new Exception("Invalid person retrieved [min=" + salary + ", max=" + maxSalary +
+                        ", person=" + p + ']');
+        }
+
+        return true;
+    }
+
+    /**
+     * @param minSalary Min salary.
+     * @param maxSalary Max salary.
+     * @return Query result.
+     * @throws Exception If failed.
+     */
+    private Collection<Map.Entry<Integer, Person>> executeQuery(double minSalary, double maxSalary) throws Exception {
+        CacheQuery<Map.Entry<Integer, Person>> q = (CacheQuery<Map.Entry<Integer, Person>>)qry;
+
+        return q.execute(minSalary, maxSalary).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("query");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmarkIgnite.java
new file mode 100644
index 0000000..4405afa
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmarkIgnite.java
@@ -0,0 +1,118 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.yardstick.cache.model.*;
+import org.yardstickframework.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.yardstickframework.BenchmarkUtils.*;
+
+/**
+ * GridGain benchmark that performs query operations with joins.
+ */
+public class IgniteSqlQueryJoinBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** */
+    private CacheQuery qry;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        println(cfg, "Populating query data...");
+
+        long start = System.nanoTime();
+
+        try (IgniteDataLoader<Object, Object> dataLdr = grid().dataLoader(cache.getName())) {
+            final int orgRange = args.range() / 10;
+
+            // Populate organizations.
+            for (int i = 0; i < orgRange && !Thread.currentThread().isInterrupted(); i++)
+                dataLdr.addData(i, new Organization(i, "org" + i));
+
+            dataLdr.flush();
+
+            // Populate persons.
+            for (int i = 0; i < args.range() && !Thread.currentThread().isInterrupted(); i++) {
+                Person p =
+                    new Person(i, ThreadLocalRandom.current().nextInt(orgRange), "firstName" + i, "lastName" + i, i * 1000);
+
+                dataLdr.addData(i, p);
+
+                if (i % 100000 == 0)
+                    println(cfg, "Populated persons: " + i);
+            }
+        }
+
+        println(cfg, "Finished populating join query data in " + ((System.nanoTime() - start) / 1_000_000) + " ms.");
+
+        qry = null; // TODO: should be fixed after IGNITE-2 cache.queries().createSqlFieldsQuery(
+            // "select p.id, p.orgId, p.firstName, p.lastName, p.salary, o.name " +
+            //    "from Person p, Organization o " +
+            //    "where p.id = o.id and salary >= ? and salary <= ?");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        double salary = ThreadLocalRandom.current().nextDouble() * args.range() * 1000;
+
+        double maxSalary = salary + 1000;
+
+        Collection<List<?>> lists = executeQueryJoin(salary, maxSalary);
+
+        for (List<?> l : lists) {
+            double sal = (Double)l.get(4);
+
+            if (sal < salary || sal > maxSalary) {
+                Person p = new Person();
+
+                p.setId((Integer)l.get(0));
+                p.setOrganizationId((Integer)l.get(1));
+                p.setFirstName((String)l.get(2));
+                p.setLastName((String)l.get(3));
+                p.setSalary(sal);
+
+                throw new Exception("Invalid person retrieved [min=" + salary + ", max=" + maxSalary +
+                    ", person=" + p + ']');
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * @param minSalary Min salary.
+     * @param maxSalary Max salary.
+     * @return Query results.
+     * @throws Exception If failed.
+     */
+    private Collection<List<?>> executeQueryJoin(double minSalary, double maxSalary) throws Exception {
+        CacheQuery<List<?>> q = (CacheQuery<List<?>>)qry;
+
+        return q.execute(minSalary, maxSalary).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("query");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmarkIgnite.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmarkIgnite.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmarkIgnite.java
new file mode 100644
index 0000000..8bef1e5
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmarkIgnite.java
@@ -0,0 +1,86 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.yardstick.cache.model.*;
+import org.yardstickframework.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * GridGain benchmark that performs put and query operations.
+ */
+public class IgniteSqlQueryPutBenchmarkIgnite extends IgniteCacheAbstractBenchmark {
+    /** */
+    private CacheQuery qry;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        qry = null; // TODO: should be fixed after IGNITE-2 cache.queries().createSqlQuery(Person.class, "salary >= ? and salary <= ?");
+    }
+
+    /** {@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;
+
+            Collection<Map.Entry<Integer, Person>> entries = executeQuery(salary, maxSalary);
+
+            for (Map.Entry<Integer, Person> entry : entries) {
+                Person p = entry.getValue();
+
+                if (p.getSalary() < salary || p.getSalary() > maxSalary)
+                    throw new Exception("Invalid person retrieved [min=" + salary + ", max=" + maxSalary +
+                            ", person=" + p + ']');
+            }
+        }
+        else {
+            int i = rnd.nextInt(args.range());
+
+            cache.put(i, new Person(i, "firstName" + i, "lastName" + i, i * 1000));
+        }
+
+        return true;
+    }
+
+    /**
+     * @param minSalary Min salary.
+     * @param maxSalary Max salary.
+     * @return Query result.
+     * @throws Exception If failed.
+     */
+    private Collection<Map.Entry<Integer, Person>> executeQuery(double minSalary, double maxSalary) throws Exception {
+        CacheQuery<Map.Entry<Integer, Person>> q = (CacheQuery<Map.Entry<Integer, Person>>)qry;
+
+        return q.execute(minSalary, maxSalary).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return grid().jcache("query");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Organization.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Organization.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Organization.java
new file mode 100644
index 0000000..b0c3e8d
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Organization.java
@@ -0,0 +1,110 @@
+/*
+ * 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.model;
+
+import org.apache.ignite.cache.query.*;
+
+import java.io.*;
+
+/**
+ * Organization record used for query test.
+ */
+public class Organization implements Externalizable {
+    /** Organization ID. */
+    @CacheQuerySqlField(index = true)
+    private int id;
+
+    /** Organization name. */
+    @CacheQuerySqlField(index = true)
+    private String name;
+
+    /**
+     * Constructs empty organization.
+     */
+    public Organization() {
+        // No-op.
+    }
+
+    /**
+     * Constructs organization with given ID.
+     *
+     * @param id Organization ID.
+     * @param name Organization name.
+     */
+    public Organization(int id, String name) {
+        this.id = id;
+        this.name = name;
+    }
+
+    /**
+     * @return Organization id.
+     */
+    public int getId() {
+        return id;
+    }
+
+    /**
+     * @param id Organization id.
+     */
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    /**
+     * @return Organization name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @param name Organization name.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(id);
+        out.writeUTF(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        id = in.readInt();
+        name = in.readUTF();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        return this == o || (o instanceof Organization) && id == ((Organization)o).id;
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Organization [id=" + id +
+            ", name=" + name + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person.java
new file mode 100644
index 0000000..fd7cd73
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person.java
@@ -0,0 +1,191 @@
+/*
+ * 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.model;
+
+import org.apache.ignite.cache.query.*;
+
+import java.io.*;
+
+/**
+ * Person record used for query test.
+ */
+public class Person implements Externalizable {
+    /** Person ID. */
+    @CacheQuerySqlField(index = true)
+    private int id;
+
+    /** Organization ID. */
+    @CacheQuerySqlField(index = true)
+    private int orgId;
+
+    /** First name (not-indexed). */
+    @CacheQuerySqlField
+    private String firstName;
+
+    /** Last name (not indexed). */
+    @CacheQuerySqlField
+    private String lastName;
+
+    /** Salary. */
+    @CacheQuerySqlField(index = true)
+    private double salary;
+
+    /**
+     * Constructs empty person.
+     */
+    public Person() {
+        // No-op.
+    }
+
+    /**
+     * Constructs person record that is not linked to any organization.
+     *
+     * @param id Person ID.
+     * @param firstName First name.
+     * @param lastName Last name.
+     * @param salary Salary.
+     */
+    public Person(int id, String firstName, String lastName, double salary) {
+        this(id, 0, firstName, lastName, salary);
+    }
+
+    /**
+     * Constructs person record.
+     *
+     * @param id Person ID.
+     * @param orgId Organization ID.
+     * @param firstName First name.
+     * @param lastName Last name.
+     * @param salary Salary.
+     */
+    public Person(int id, int orgId, String firstName, String lastName, double salary) {
+        this.id = id;
+        this.orgId = orgId;
+        this.firstName = firstName;
+        this.lastName = lastName;
+        this.salary = salary;
+    }
+
+    /**
+     * @return Person id.
+     */
+    public int getId() {
+        return id;
+    }
+
+    /**
+     * @param id Person id.
+     */
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    /**
+     * @return Organization id.
+     */
+    public int getOrganizationId() {
+        return orgId;
+    }
+
+    /**
+     * @param orgId Organization id.
+     */
+    public void setOrganizationId(int orgId) {
+        this.orgId = orgId;
+    }
+
+    /**
+     * @return Person first name.
+     */
+    public String getFirstName() {
+        return firstName;
+    }
+
+    /**
+     * @param firstName Person first name.
+     */
+    public void setFirstName(String firstName) {
+        this.firstName = firstName;
+    }
+
+    /**
+     * @return Person last name.
+     */
+    public String getLastName() {
+        return lastName;
+    }
+
+    /**
+     * @param lastName Person last name.
+     */
+    public void setLastName(String lastName) {
+        this.lastName = lastName;
+    }
+
+    /**
+     * @return Salary.
+     */
+    public double getSalary() {
+        return salary;
+    }
+
+    /**
+     * @param salary Salary.
+     */
+    public void setSalary(double salary) {
+        this.salary = salary;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(id);
+        out.writeInt(orgId);
+        out.writeUTF(firstName);
+        out.writeUTF(lastName);
+        out.writeDouble(salary);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        id = in.readInt();
+        orgId = in.readInt();
+        firstName = in.readUTF();
+        lastName = in.readUTF();
+        salary = in.readDouble();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        return this == o || (o instanceof Person) && id == ((Person)o).id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Person [firstName=" + firstName +
+            ", id=" + id +
+            ", orgId=" + orgId +
+            ", lastName=" + lastName +
+            ", salary=" + salary +
+            ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
new file mode 100644
index 0000000..b6cb5d1
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/SampleValue.java
@@ -0,0 +1,62 @@
+/*
+ * 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.model;
+
+import java.io.*;
+
+/**
+ * Entity class for benchmark.
+ */
+public class SampleValue implements Externalizable {
+    /** */
+    private int id;
+
+    /** */
+    public SampleValue() {
+        // No-op.
+    }
+
+    /**
+     * @param id Id.
+     */
+    public SampleValue(int id) {
+        this.id = id;
+    }
+
+    /**
+     * @return Id.
+     */
+    public int id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        id = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(id);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Value [id=" + id + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteAffinityCallBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteAffinityCallBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteAffinityCallBenchmark.java
new file mode 100644
index 0000000..28ac816
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteAffinityCallBenchmark.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.compute;
+
+import org.apache.ignite.yardstick.*;
+import org.apache.ignite.yardstick.compute.model.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * GridGain benchmark that performs affinity call operations.
+ */
+public class IgniteAffinityCallBenchmark extends IgniteAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        grid().compute().affinityCall("compute", ThreadLocalRandom.current().nextInt(), new NoopCallable());
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteApplyBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteApplyBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteApplyBenchmark.java
new file mode 100644
index 0000000..0b40cdd
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteApplyBenchmark.java
@@ -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.
+ */
+
+package org.apache.ignite.yardstick.compute;
+
+import org.apache.ignite.lang.*;
+import org.apache.ignite.yardstick.*;
+import org.yardstickframework.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs apply operations.
+ */
+public class IgniteApplyBenchmark extends IgniteAbstractBenchmark {
+    /** Args for apply. */
+    private List<Integer> applyArgs;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        assert args.jobs() > 0;
+
+        applyArgs = new ArrayList<>(args.jobs());
+
+        for (int i = 0; i < args.jobs(); ++i)
+            applyArgs.add(null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        grid().compute().apply(new NoopClosure(), applyArgs);
+
+        return true;
+    }
+
+    /**
+     *
+     */
+    public static class NoopClosure implements IgniteClosure<Integer, Object>, Externalizable {
+        /** {@inheritDoc} */
+        @Override public Object apply(Integer o) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            //No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            //No-op
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteBroadcastBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteBroadcastBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteBroadcastBenchmark.java
new file mode 100644
index 0000000..0e37efa
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteBroadcastBenchmark.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.compute;
+
+import org.apache.ignite.yardstick.*;
+import org.apache.ignite.yardstick.compute.model.*;
+
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs broadcast operations.
+ */
+public class IgniteBroadcastBenchmark extends IgniteAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        grid().compute().broadcast(new NoopCallable());
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteExecuteBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteExecuteBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteExecuteBenchmark.java
new file mode 100644
index 0000000..4fc4530
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteExecuteBenchmark.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.compute;
+
+import org.apache.ignite.yardstick.*;
+import org.apache.ignite.yardstick.compute.model.*;
+
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs execute operations.
+ */
+public class IgniteExecuteBenchmark extends IgniteAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        grid().compute().execute(new NoopTask(args.jobs()), null);
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteRunBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteRunBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteRunBenchmark.java
new file mode 100644
index 0000000..d0ca63e
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/IgniteRunBenchmark.java
@@ -0,0 +1,71 @@
+/*
+ * 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.compute;
+
+import org.apache.ignite.yardstick.*;
+import org.yardstickframework.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * GridGain benchmark that performs run operations.
+ */
+public class IgniteRunBenchmark extends IgniteAbstractBenchmark {
+    /** Jobs for run */
+    private List<Runnable> jobs;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        assert args.jobs() > 0;
+
+        jobs = new ArrayList<>(args.jobs());
+
+        for (int i = 0; i < args.jobs(); ++i)
+            jobs.add(new NoopRunnable());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        grid().compute().run(jobs);
+        
+        return true;
+    }
+
+    /**
+     *
+     */
+    public static class NoopRunnable implements Runnable, Externalizable {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            //No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            //No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            //No-op
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopCallable.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopCallable.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopCallable.java
new file mode 100644
index 0000000..5a32618
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopCallable.java
@@ -0,0 +1,41 @@
+/*
+ * 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.compute.model;
+
+import java.io.*;
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class NoopCallable implements Callable<Object>, Externalizable {
+    /** {@inheritDoc} */
+    @Override public Object call() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        //No-op
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        //No-op
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopTask.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopTask.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopTask.java
new file mode 100644
index 0000000..34ab2ad
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/compute/model/NoopTask.java
@@ -0,0 +1,97 @@
+/*
+ * 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.compute.model;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Assigns {@link NoopJob} job for each node.
+ */
+public class NoopTask implements ComputeTask<Object, Object> {
+    /** Number of jobs */
+    private int jobs;
+
+    /**
+     * @param jobs Number of jobs
+     */
+    public NoopTask(int jobs) {
+        assert jobs > 0;
+
+        this.jobs = jobs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ComputeJobResultPolicy result(
+        ComputeJobResult res,
+        List<ComputeJobResult> rcvd
+    ) throws IgniteCheckedException {
+        return ComputeJobResultPolicy.WAIT;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(
+        List<ClusterNode> subgrid,
+        @Nullable Object arg
+    ) throws IgniteCheckedException {
+        Map<ComputeJob, ClusterNode> map = new HashMap<>((int)(subgrid.size() * jobs / 0.75));
+
+        for (ClusterNode gridNode : subgrid) {
+            //assigns jobs for each node
+            for (int i = 0; i < jobs; ++i)
+                map.put(new NoopJob(), gridNode);
+        }
+
+        return map;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+        return null;
+    }
+
+    /**
+     *
+     */
+    public static class NoopJob implements ComputeJob, Externalizable {
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() throws IgniteCheckedException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            //No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            //No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            //No-op
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0031ccd/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b071a30..85d8321 100644
--- a/pom.xml
+++ b/pom.xml
@@ -15,7 +15,7 @@
   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.
-  -->
+-->
 
 <!--
     POM file.
@@ -1264,5 +1264,16 @@
                 </plugins>
             </build>
         </profile>
+
+        <profile>
+            <id>ignite-yardstick</id>
+            <activation>
+                <activeByDefault>false</activeByDefault>
+            </activation>
+
+            <modules>
+                <module>modules/yardstick</module>
+            </modules>
+        </profile>
     </profiles>
 </project>