You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/02/02 09:33:16 UTC

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

Repository: incubator-ignite
Updated Branches:
  refs/heads/sprint-1 7e8ea4f1d -> 06d9b7d23


# 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/sprint-1
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>


[10/12] incubator-ignite git commit: # IGNITE-32: Fixed profile.

Posted by ak...@apache.org.
# IGNITE-32: Fixed profile.


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

Branch: refs/heads/sprint-1
Commit: b82eea32318a786cc5f5d352d18981220adf77ee
Parents: 8112a50
Author: AKuznetsov <ak...@gridgain.com>
Authored: Thu Jan 29 13:50:02 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Thu Jan 29 13:50:02 2015 +0700

----------------------------------------------------------------------
 pom.xml | 6 +-----
 1 file changed, 1 insertion(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b82eea32/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4f64724..06c9d42 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1280,11 +1280,7 @@
         </profile>
 
         <profile>
-            <id>ignite-yardstick</id>
-            <activation>
-                <activeByDefault>false</activeByDefault>
-            </activation>
-
+            <id>benchmarks</id>
             <modules>
                 <module>modules/yardstick</module>
             </modules>


[09/12] incubator-ignite git commit: # IGNITE-102: Remove debug.

Posted by ak...@apache.org.
# IGNITE-102: Remove debug.


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

Branch: refs/heads/sprint-1
Commit: 8112a5062adcd0931caafd166c0e2bf6e2b2674e
Parents: fa4ac59
Author: AKuznetsov <ak...@gridgain.com>
Authored: Thu Jan 29 10:27:23 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Thu Jan 29 10:27:23 2015 +0700

----------------------------------------------------------------------
 .../ignite/yardstick/cache/IgnitePutGetTxBenchmark.java      | 8 --------
 1 file changed, 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8112a506/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
index 944d6a4..206dabc 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
@@ -31,23 +31,15 @@ public class IgnitePutGetTxBenchmark extends IgniteCacheAbstractBenchmark {
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
         int key = nextRandom(0, args.range() / 2);
 
-        ignite().log().debug("IgnitePutGetTxBenchmark - key: " + key);
-
         try (IgniteTx tx = ignite().transactions().txStart()) {
-            ignite().log().debug("IgnitePutGetTxBenchmark: txStart()");
-
             Object val = cache.get(key);
 
-            ignite().log().debug("IgnitePutGetTxBenchmark - get():" + val);
-
             if (val != null)
                 key = nextRandom(args.range() / 2, args.range());
 
             cache.put(key, new SampleValue(key));
 
             tx.commit();
-
-            ignite().log().debug("IgnitePutGetTxBenchmark - txCommit");
         }
 
         return true;


[12/12] incubator-ignite git commit: # IGNITE-102: Merge with sprint-1.

Posted by ak...@apache.org.
# IGNITE-102: Merge with sprint-1.


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

Branch: refs/heads/sprint-1
Commit: 06d9b7d2396a83566a131e3959f894bf9e08e865
Parents: 6f49e95
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Feb 2 15:32:24 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Feb 2 15:32:24 2015 +0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/yardstick/IgniteNode.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06d9b7d2/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
index ad71ca2..b07d1f0 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -134,7 +134,7 @@ public class IgniteNode implements BenchmarkServer {
             url = new URL(springCfgPath);
         }
         catch (MalformedURLException e) {
-            url = GridUtils.resolveGridGainUrl(springCfgPath);
+            url = IgniteUtils.resolveGridGainUrl(springCfgPath);
 
             if (url == null)
                 throw new IgniteCheckedException("Spring XML configuration path is invalid: " + springCfgPath +


[05/12] incubator-ignite git commit: Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102

Posted by ak...@apache.org.
Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102


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

Branch: refs/heads/sprint-1
Commit: 786193fead9e530f60eeda0f1a898b5d2970f536
Parents: aee11a0 d97965b
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jan 28 16:08:47 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jan 28 16:08:47 2015 +0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/cache/CacheManager.java | 10 +++++++++-
 .../org/apache/ignite/IgniteCacheAffinityTest.java | 17 +++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java    |  1 +
 3 files changed, 27 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[11/12] incubator-ignite git commit: Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102

Posted by ak...@apache.org.
Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102


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

Branch: refs/heads/sprint-1
Commit: 6f49e95540bcf0be76e82a635e1420d666c7a24d
Parents: b82eea3 7e8ea4f
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Feb 2 15:13:37 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Feb 2 15:13:37 2015 +0700

----------------------------------------------------------------------
 assembly/libs/readme.txt                        |   115 +-
 assembly/release-base-fabric.xml                |     6 +-
 assembly/release-base.xml                       |     6 +-
 assembly/release-hadoop.xml                     |     6 +-
 bin/ggrouter.bat                                |    33 -
 bin/ggrouter.sh                                 |    50 -
 bin/ggstart.bat                                 |   228 -
 bin/ggstart.sh                                  |   176 -
 bin/ggvisorcmd.bat                              |   144 -
 bin/ggvisorcmd.sh                               |   124 -
 bin/ignite.bat                                  |   228 +
 bin/ignite.sh                                   |   176 +
 bin/igniterouter.bat                            |    33 +
 bin/igniterouter.sh                             |    50 +
 bin/ignitevisorcmd.bat                          |   144 +
 bin/ignitevisorcmd.sh                           |   124 +
 bin/include/functions.sh                        |    42 +-
 bin/include/hadoop-classpath.bat                |     2 +-
 bin/include/hadoop-classpath.sh                 |     4 +-
 bin/include/parseargs.bat                       |     2 +-
 bin/include/parseargs.sh                        |     6 +-
 bin/include/service.sh                          |    12 +-
 bin/include/setenv.bat                          |    28 +-
 bin/include/setenv.sh                           |    34 +-
 bin/include/target-classpath.bat                |     8 +-
 bin/include/target-classpath.sh                 |    14 +-
 bin/setup-hadoop.bat                            |     2 +-
 bin/setup-hadoop.sh                             |    12 +-
 config/gridgain-log4j.xml                       |   152 -
 config/hadoop/default-config.xml                |     2 +-
 config/ignite-log4j.xml                         |   152 +
 config/java.util.logging.properties             |    16 +-
 config/router/default-router.xml                |     2 +-
 docs/core-site.gridgain.xml                     |    89 -
 docs/core-site.ignite.xml                       |    89 +
 docs/gridgain_readme.md                         |   149 -
 docs/gridgain_readme.pdf                        |   Bin 407698 -> 0 bytes
 docs/hadoop_readme.md                           |    84 +-
 docs/hadoop_readme.pdf                          |   Bin 411788 -> 82297 bytes
 docs/hive-site.gridgain.xml                     |    36 -
 docs/hive-site.ignite.xml                       |    36 +
 docs/ignite_readme.md                           |   100 +
 docs/ignite_readme.pdf                          |   Bin 0 -> 76810 bytes
 docs/mapred-site.gridgain.xml                   |    79 -
 docs/mapred-site.ignite.xml                     |    79 +
 examples/config/example-cache.xml               |     2 +-
 examples/config/example-streamer.xml            |     8 +-
 examples/config/filesystem/example-ggfs.xml     |     2 +-
 examples/config/filesystem/readme.txt           |     4 +-
 examples/config/hibernate/readme.txt            |     4 +-
 examples/config/servlet/WEB-INF/web.xml         |     4 +-
 examples/config/servlet/readme.txt              |     2 +-
 examples/rest/http-rest-example.php             |     2 +-
 examples/rest/memcache-rest-example.php         |     2 +-
 .../compute/ComputeBroadcastExample.java        |     2 +-
 .../compute/ComputeCallableExample.java         |     2 +-
 .../examples/compute/ComputeClosureExample.java |     2 +-
 .../compute/ComputeContinuousMapperExample.java |     2 +-
 .../compute/ComputeExecutorServiceExample.java  |     2 +-
 .../ComputeFibonacciContinuationExample.java    |    13 +-
 .../compute/ComputeProjectionExample.java       |     2 +-
 .../examples/compute/ComputeReducerExample.java |     2 +-
 .../compute/ComputeRunnableExample.java         |     7 +-
 .../compute/ComputeScheduleExample.java         |     2 +-
 .../examples/compute/ComputeTaskMapExample.java |     2 +-
 .../compute/ComputeTaskSplitExample.java        |     2 +-
 .../compute/montecarlo/CreditRiskExample.java   |     2 +-
 .../examples/datagrid/CacheAffinityExample.java |     2 +-
 .../examples/datagrid/CacheApiExample.java      |     2 +-
 .../datagrid/CacheContinuousQueryExample.java   |     2 +-
 .../datagrid/CacheDataLoaderExample.java        |     2 +-
 .../examples/datagrid/CacheEventsExample.java   |     2 +-
 .../datagrid/CachePopularNumbersExample.java    |     2 +-
 .../examples/datagrid/CachePutGetExample.java   |     2 +-
 .../examples/datagrid/CacheQueryExample.java    |     2 +-
 .../datagrid/CacheTransactionExample.java       |     2 +-
 .../datastructures/CacheAtomicLongExample.java  |     2 +-
 .../CacheAtomicReferenceExample.java            |     2 +-
 .../CacheAtomicSequenceExample.java             |     2 +-
 .../CacheAtomicStampedExample.java              |     2 +-
 .../CacheCountDownLatchExample.java             |     2 +-
 .../datastructures/CacheQueueExample.java       |     2 +-
 .../datastructures/CacheSetExample.java         |     2 +-
 .../hibernate/HibernateL2CacheExample.java      |     2 +-
 .../starschema/CacheStarSchemaExample.java      |     2 +-
 .../ignite/examples/events/EventsExample.java   |     2 +-
 .../ignite/examples/ggfs/GgfsExample.java       |     2 +-
 .../examples/ggfs/GgfsMapReduceExample.java     |     2 +-
 .../ignite/examples/ggfs/GgfsNodeStartup.java   |     4 +-
 .../examples/messaging/MessagingExample.java    |     2 +-
 .../messaging/MessagingPingPongExample.java     |     2 +-
 .../MessagingPingPongListenActorExample.java    |     2 +-
 .../misc/deployment/DeploymentExample.java      |     2 +-
 .../misc/springbean/SpringBeanExample.java      |     2 +-
 .../examples/services/ServicesExample.java      |     2 +-
 .../streaming/StreamingCheckInExample.java      |     8 +-
 .../StreamingPopularNumbersExample.java         |     8 +-
 .../streaming/StreamingPriceBarsExample.java    |     8 +-
 .../StreamingRunningAverageExample.java         |     8 +-
 .../apache/ignite/examples/ComputeExample.java  |    59 +
 .../ignite/examples/MessagingExample.java       |   166 +
 .../org/gridgain/examples/ComputeExample.java   |    59 -
 .../org/gridgain/examples/MessagingExample.java |   166 -
 .../examples/ScalarCacheAffinityExample1.scala  |     2 +-
 .../examples/ScalarCacheAffinityExample2.scala  |     2 +-
 .../ScalarCacheAffinitySimpleExample.scala      |     2 +-
 .../scalar/examples/ScalarCacheExample.scala    |     2 +-
 .../ScalarCachePopularNumbersExample.scala      |     2 +-
 .../examples/ScalarCacheQueryExample.scala      |     2 +-
 .../scalar/examples/ScalarClosureExample.scala  |     2 +-
 .../examples/ScalarContinuationExample.scala    |    11 +-
 .../examples/ScalarCreditRiskExample.scala      |     2 +-
 .../examples/ScalarPiCalculationExample.scala   |     2 +-
 .../scalar/examples/ScalarPingPongExample.scala |     2 +-
 .../scalar/examples/ScalarPrimeExample.scala    |     2 +-
 .../scalar/examples/ScalarScheduleExample.scala |     2 +-
 .../examples/ScalarSnowflakeSchemaExample.scala |     2 +-
 .../scalar/examples/ScalarTaskExample.scala     |     2 +-
 .../examples/ScalarWorldShortestMapReduce.scala |     2 +-
 .../GridMonteCarloExamplesSelfTest.java         |     4 +-
 .../testsuites/IgniteExamplesSelfTestSuite.java |     2 +-
 .../ScalarExamplesSelfTestSuite.scala           |     2 +-
 ipc/readme.txt                                  |     4 +-
 ipc/shmem/Makefile.am                           |     2 +-
 ipc/shmem/Makefile.in                           |     2 +-
 ipc/shmem/configure                             |     4 +-
 ipc/shmem/configure.ac                          |     2 +-
 ipc/shmem/ggshmem/Makefile.am                   |    23 -
 ipc/shmem/ggshmem/Makefile.in                   |   589 -
 ..._util_ipc_shmem_GridIpcSharedMemoryUtils.cpp |   882 --
 ipc/shmem/igniteshmem/Makefile.am               |    23 +
 ipc/shmem/igniteshmem/Makefile.in               |   589 +
 ...rnal_util_ipc_shmem_IpcSharedMemoryUtils.cpp |   882 ++
 ipc/shmem/include/Makefile.am                   |     2 +-
 ipc/shmem/include/Makefile.in                   |     2 +-
 ...ternal_util_ipc_shmem_IpcSharedMemoryUtils.h |   117 +
 ...id_util_ipc_shmem_GridIpcSharedMemoryUtils.h |   117 -
 ipc/shmem/readme.txt                            |    17 +-
 modules/aop/readme.txt                          |    14 +-
 .../GridifySetToXXXNonSpringAopSelfTest.java    |    20 +-
 .../GridifySetToXXXSpringAopSelfTest.java       |    20 +-
 .../ignite/gridify/NonSpringAopSelfTest.java    |    20 +-
 .../optimized/OptimizedMarshallerAopTest.java   |     4 +-
 .../testsuites/IgniteAopSelfTestSuite.java      |     2 +-
 .../gridify/ExternalNonSpringAopSelfTest.java   |    20 +-
 modules/aws/readme.txt                          |    14 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |     4 +-
 modules/clients/src/test/bin/start-nodes-ssh.sh |     2 +-
 modules/clients/src/test/bin/start-nodes.cmd    |    24 +-
 modules/clients/src/test/bin/start-nodes.sh     |    34 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |     8 +-
 .../client/ClientDefaultCacheSelfTest.java      |     4 +-
 .../client/ClientTopologyCacheSelfTest.java     |     2 +-
 .../ClientPropertiesConfigurationSelfTest.java  |     4 +-
 .../ClientAbstractMultiNodeSelfTest.java        |     4 +-
 .../integration/ClientAbstractSelfTest.java     |    14 +-
 .../client/router/ClientFailedInitSelfTest.java |     4 +-
 .../client/router/RouterFactorySelfTest.java    |     4 +-
 .../JettyRestProcessorAbstractSelfTest.java     |     4 +-
 .../rest/RestBinaryProtocolSelfTest.java        |     4 +-
 .../rest/RestProcessorStartSelfTest.java        |     3 +-
 .../rest/TaskCommandHandlerSelfTest.java        |     6 +-
 .../protocols/tcp/TcpRestParserSelfTest.java    |     6 +-
 .../src/test/resources/jetty/rest-jetty-ssl.xml |    10 +-
 .../src/test/resources/jetty/rest-jetty.xml     |     8 +-
 .../test/resources/jetty/router-jetty-ssl.xml   |     8 +-
 .../src/test/resources/jetty/router-jetty.xml   |     8 +-
 modules/clients/src/test/resources/log4j.xml    |     4 +-
 .../clients/src/test/resources/spring-cache.xml |     2 +-
 .../src/test/resources/spring-router-ssl.xml    |     4 +-
 .../src/test/resources/spring-router.xml        |     4 +-
 .../src/test/resources/spring-server-node.xml   |     2 +-
 .../test/resources/spring-server-ssl-node.xml   |     2 +-
 .../java/META-INF/native/linux64/libggshmem.so  |   Bin 138023 -> 0 bytes
 .../META-INF/native/linux64/libigniteshmem.so   |   Bin 0 -> 138345 bytes
 .../java/META-INF/native/osx/libggshmem.dylib   |   Bin 32940 -> 0 bytes
 .../META-INF/native/osx/libigniteshmem.dylib    |   Bin 0 -> 33116 bytes
 .../apache/ignite/IgniteBasicWarmupClosure.java |     6 +-
 .../apache/ignite/IgniteCheckedException.java   |     2 +-
 .../java/org/apache/ignite/IgniteCluster.java   |     6 +-
 .../org/apache/ignite/IgniteDataLoader.java     |    14 +-
 .../java/org/apache/ignite/IgniteException.java |     2 +-
 .../main/java/org/apache/ignite/IgniteFs.java   |     4 +-
 .../java/org/apache/ignite/IgniteLogger.java    |     6 +-
 .../java/org/apache/ignite/IgniteScheduler.java |     6 +-
 .../apache/ignite/IgniteSystemProperties.java   |   198 +-
 .../main/java/org/apache/ignite/Ignition.java   |    54 +-
 .../apache/ignite/cache/CacheConfiguration.java |     3 +-
 .../org/apache/ignite/cache/CacheEntry.java     |    27 +-
 .../apache/ignite/cache/CacheProjection.java    |    43 +-
 .../apache/ignite/cache/CachingProvider.java    |     2 +-
 .../java/org/apache/ignite/cache/GridCache.java |     5 +-
 .../CacheConsistentHashAffinityFunction.java    |     2 +-
 .../apache/ignite/cache/query/CacheQueries.java |     7 +-
 .../apache/ignite/cache/query/CacheQuery.java   |     3 +-
 .../ignite/cache/query/CacheQueryFuture.java    |     4 +-
 .../cache/query/CacheQuerySqlFunction.java      |     4 +-
 .../cache/query/annotations/QuerySqlField.java  |     2 +-
 .../query/annotations/QuerySqlFunction.java     |     2 +-
 .../apache/ignite/cache/store/CacheStore.java   |     2 +-
 .../apache/ignite/client/GridClientCompute.java |     8 +-
 .../ignite/client/router/GridRouterFactory.java |     4 +-
 .../impl/GridRouterCommandLineStartup.java      |     4 +-
 .../router/impl/GridTcpRouterNioParser.java     |     4 +-
 .../ignite/compute/ComputeTaskFuture.java       |     4 +-
 .../ignite/compute/ComputeTaskSession.java      |     3 +-
 .../apache/ignite/compute/gridify/Gridify.java  |     2 +-
 .../ignite/compute/gridify/GridifySetToSet.java |     4 +-
 .../compute/gridify/GridifySetToValue.java      |     4 +-
 .../ClientConnectionConfiguration.java          |    16 +-
 .../configuration/GridQueryConfiguration.java   |   201 -
 .../configuration/IgniteConfiguration.java      |    80 +-
 .../configuration/IgniteDeploymentMode.java     |     4 +-
 .../configuration/IgniteQueryConfiguration.java |   201 +
 .../apache/ignite/fs/IgniteFsConfiguration.java |     2 +-
 .../ignite/fs/mapreduce/IgniteFsTask.java       |     2 +-
 .../org/apache/ignite/hadoop/GridHadoop.java    |     6 +-
 .../ignite/internal/ClusterGroupAdapter.java    |     2 +-
 .../internal/ClusterNodeLocalMapImpl.java       |     2 +-
 .../java/org/apache/ignite/internal/GridEx.java |   143 -
 .../org/apache/ignite/internal/GridGainEx.java  |  2399 ---
 .../ignite/internal/GridJobSessionImpl.java     |     2 +-
 .../org/apache/ignite/internal/GridKernal.java  |  3322 ----
 .../ignite/internal/GridKernalContext.java      |     2 +-
 .../ignite/internal/GridKernalContextImpl.java  |    10 +-
 .../apache/ignite/internal/GridLoggerProxy.java |     6 +-
 .../internal/GridPerformanceSuggestions.java    |     4 +-
 .../apache/ignite/internal/GridProductImpl.java |     2 +-
 .../ignite/internal/GridTaskSessionImpl.java    |     2 +-
 .../ignite/internal/GridUpdateNotifier.java     |     2 +-
 .../ignite/internal/IgniteClusterAsyncImpl.java |     4 +-
 .../org/apache/ignite/internal/IgniteEx.java    |   143 +
 .../ignite/internal/IgniteInternalFuture.java   |   190 +
 .../apache/ignite/internal/IgniteKernal.java    |  3322 ++++
 .../ignite/internal/IgniteSchedulerImpl.java    |     6 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  2396 +++
 .../internal/executor/GridExecutorService.java  |    44 +-
 .../discovery/GridDiscoveryManager.java         |     2 +-
 .../eventstorage/GridEventStorageManager.java   |     4 +-
 .../affinity/GridAffinityAssignmentCache.java   |     5 +-
 .../affinity/GridAffinityProcessor.java         |     8 +-
 .../processors/affinity/GridAffinityUtils.java  |     2 +-
 .../processors/cache/CacheLockImpl.java         |     4 +-
 .../processors/cache/GridCacheAdapter.java      |   366 +-
 .../cache/GridCacheAffinityManager.java         |     8 +-
 .../processors/cache/GridCacheContext.java      |    16 +-
 .../cache/GridCacheDeploymentManager.java       |     2 +-
 .../processors/cache/GridCacheEntryImpl.java    |    27 +-
 .../cache/GridCacheEvictionEntry.java           |    27 +-
 .../cache/GridCacheEvictionManager.java         |     4 +-
 .../cache/GridCacheExplicitLockSpan.java        |     4 +-
 .../cache/GridCacheFilterEvaluationEntry.java   |    27 +-
 .../processors/cache/GridCacheFuture.java       |     3 +-
 .../processors/cache/GridCacheIoManager.java    |    15 +-
 .../cache/GridCacheMultiTxFuture.java           |     6 +-
 .../cache/GridCacheMvccCandidate.java           |     4 +-
 .../processors/cache/GridCacheMvccManager.java  |    29 +-
 .../GridCachePartitionExchangeManager.java      |    11 +-
 .../processors/cache/GridCachePreloader.java    |     7 +-
 .../cache/GridCachePreloaderAdapter.java        |     9 +-
 .../processors/cache/GridCacheProcessor.java    |    13 +-
 .../processors/cache/GridCacheProjectionEx.java |    27 +-
 .../cache/GridCacheProjectionImpl.java          |    69 +-
 .../processors/cache/GridCacheProxyImpl.java    |    73 +-
 .../cache/GridCacheSharedContext.java           |    13 +-
 .../processors/cache/GridCacheStoreManager.java |    25 +-
 .../processors/cache/GridCacheUtils.java        |     9 +-
 .../processors/cache/IgniteCacheProxy.java      |    37 +-
 .../GridCacheAtomicSequenceImpl.java            |     3 +-
 .../GridCacheDataStructuresManager.java         |     4 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |     7 +-
 ...dCachePessimisticCheckCommittedTxFuture.java |     7 +-
 .../distributed/GridCacheTxFinishSync.java      |     8 +-
 .../GridDistributedCacheAdapter.java            |     7 +-
 .../GridDistributedTxRemoteAdapter.java         |     7 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    12 +-
 .../distributed/dht/GridDhtCacheEntry.java      |     7 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |     8 +-
 .../cache/distributed/dht/GridDhtFuture.java    |     4 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    11 +-
 .../distributed/dht/GridDhtLocalPartition.java  |     7 +-
 .../distributed/dht/GridDhtLockFuture.java      |    11 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |     4 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |    29 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    11 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |    19 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    11 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    11 +-
 .../dht/GridPartitionedGetFuture.java           |    34 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   122 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |    34 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    24 +-
 .../colocated/GridDhtColocatedLockFuture.java   |    23 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |     7 +-
 .../preloader/GridDhtPartitionDemandPool.java   |    13 +-
 .../GridDhtPartitionsExchangeFuture.java        |    20 +-
 .../dht/preloader/GridDhtPreloader.java         |    29 +-
 .../distributed/near/GridNearAtomicCache.java   |    47 +-
 .../distributed/near/GridNearCacheAdapter.java  |    15 +-
 .../distributed/near/GridNearGetFuture.java     |    38 +-
 .../distributed/near/GridNearLockFuture.java    |    23 +-
 .../near/GridNearTransactionalCache.java        |     9 +-
 .../near/GridNearTxFinishFuture.java            |    17 +-
 .../cache/distributed/near/GridNearTxLocal.java |    58 +-
 .../near/GridNearTxPrepareFuture.java           |    17 +-
 .../processors/cache/local/GridLocalCache.java  |     7 +-
 .../processors/cache/local/GridLocalTx.java     |    12 +-
 .../local/atomic/GridLocalAtomicCache.java      |   139 +-
 .../GridCacheDistributedFieldsQueryFuture.java  |     4 +-
 .../query/GridCacheDistributedQueryManager.java |     9 +-
 .../query/GridCacheFieldsQueryErrorFuture.java  |     3 +-
 .../query/GridCacheLocalFieldsQueryFuture.java  |     4 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |     3 +-
 .../cache/query/GridCacheQueriesEx.java         |     6 +-
 .../cache/query/GridCacheQueriesImpl.java       |    11 +-
 .../cache/query/GridCacheQueriesProxy.java      |    11 +-
 .../cache/query/GridCacheQueryManager.java      |    50 +-
 .../query/GridCacheQueryMetadataAware.java      |     4 +-
 .../GridCacheContinuousQueryEntry.java          |    27 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |     2 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |     2 +-
 .../transactions/IgniteTransactionsImpl.java    |     3 +-
 .../cache/transactions/IgniteTxAdapter.java     |     7 +-
 .../cache/transactions/IgniteTxEx.java          |     9 +-
 .../cache/transactions/IgniteTxHandler.java     |    48 +-
 .../transactions/IgniteTxLocalAdapter.java      |   148 +-
 .../cache/transactions/IgniteTxLocalEx.java     |    17 +-
 .../cache/transactions/IgniteTxManager.java     |    25 +-
 .../cache/transactions/IgniteTxProxyImpl.java   |     9 +-
 .../closure/GridClosureProcessor.java           |    56 +-
 .../continuous/GridContinuousProcessor.java     |     4 +-
 .../dataload/GridDataLoaderProcessor.java       |     5 +-
 .../dataload/IgniteDataLoaderImpl.java          |    58 +-
 .../dr/GridDrDataLoadCacheUpdater.java          |     5 +-
 .../email/IgniteEmailProcessorAdapter.java      |     5 +-
 .../email/IgniteNoopEmailProcessor.java         |     5 +-
 .../processors/fs/GridGgfsAsyncImpl.java        |     3 +-
 .../processors/fs/GridGgfsDataManager.java      |    30 +-
 .../internal/processors/fs/GridGgfsEx.java      |     3 +-
 .../internal/processors/fs/GridGgfsImpl.java    |    18 +-
 .../processors/fs/GridGgfsInputStreamImpl.java  |    19 +-
 .../processors/fs/GridGgfsIpcHandler.java       |     4 +-
 .../processors/fs/GridGgfsMetaManager.java      |     9 +-
 .../internal/processors/fs/GridGgfsServer.java  |    34 +-
 .../processors/fs/GridGgfsServerHandler.java    |     4 +-
 .../processors/fs/GridGgfsServerManager.java    |     8 +-
 .../processors/fs/IgniteFsNoopProcessor.java    |     2 +-
 .../processors/fs/IgniteFsOutputStreamImpl.java |     3 +-
 .../processors/fs/IgniteFsProcessor.java        |    12 +-
 .../processors/fs/IgniteFsProcessorAdapter.java |     2 +-
 .../hadoop/IgniteHadoopNoopProcessor.java       |     5 +-
 .../hadoop/IgniteHadoopProcessorAdapter.java    |     5 +-
 .../processors/job/GridJobProcessor.java        |     2 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |     4 +-
 .../processors/query/GridQueryIndexing.java     |     4 +-
 .../processors/query/GridQueryProcessor.java    |    12 +-
 .../resource/GridResourceProcessor.java         |     2 +-
 .../resource/GridSpringResourceContext.java     |     2 +-
 .../processors/rest/GridRestProcessor.java      |    18 +-
 .../rest/GridRestProtocolHandler.java           |     4 +-
 .../rest/handlers/GridRestCommandHandler.java   |     4 +-
 .../handlers/cache/GridCacheCommandHandler.java |    60 +-
 .../cache/GridCacheQueryCommandHandler.java     |    12 +-
 .../handlers/log/GridLogCommandHandler.java     |     7 +-
 .../metadata/GridPortableMetadataHandler.java   |     3 +-
 .../handlers/task/GridTaskCommandHandler.java   |    12 +-
 .../top/GridTopologyCommandHandler.java         |     3 +-
 .../version/GridVersionCommandHandler.java      |     3 +-
 .../protocols/tcp/GridClientPacketType.java     |     4 +-
 .../protocols/tcp/GridMemcachedMessage.java     |     4 +-
 .../tcp/GridTcpMemcachedNioListener.java        |    17 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |     5 +-
 .../rest/protocols/tcp/GridTcpRestParser.java   |    18 +-
 .../service/GridServiceProcessor.java           |    22 +-
 .../processors/service/GridServiceProxy.java    |     2 +-
 .../streamer/GridStreamProcessor.java           |     6 +-
 .../processors/streamer/IgniteStreamerImpl.java |     4 +-
 .../internal/util/GridConcurrentFactory.java    |     3 +-
 .../internal/util/GridConfigurationFinder.java  |     4 +-
 .../internal/util/GridReflectionCache.java      |     3 +-
 .../apache/ignite/internal/util/GridUtils.java  |  9100 -----------
 .../ignite/internal/util/IgniteUtils.java       |  9141 +++++++++++
 .../util/future/GridCompoundFuture.java         |    28 +-
 .../util/future/GridCompoundIdentityFuture.java |     2 +-
 .../util/future/GridEmbeddedFuture.java         |    38 +-
 .../util/future/GridFinishedFuture.java         |    10 +-
 .../util/future/GridFinishedFutureEx.java       |     9 +-
 .../internal/util/future/GridFutureAdapter.java |    34 +-
 .../util/future/GridFutureAdapterEx.java        |    31 +-
 .../util/future/GridFutureChainListener.java    |     8 +-
 .../internal/util/io/GridUnsafeDataInput.java   |     2 +-
 .../internal/util/io/GridUnsafeDataOutput.java  |     2 +-
 .../internal/util/ipc/GridIpcEndpoint.java      |    49 -
 .../util/ipc/GridIpcEndpointBindException.java  |    47 -
 .../util/ipc/GridIpcEndpointFactory.java        |    84 -
 .../internal/util/ipc/GridIpcEndpointType.java  |    29 -
 .../util/ipc/GridIpcServerEndpoint.java         |    73 -
 .../ipc/GridIpcServerEndpointDeserializer.java  |    66 -
 .../internal/util/ipc/GridIpcToNioAdapter.java  |   250 -
 .../ignite/internal/util/ipc/IpcEndpoint.java   |    49 +
 .../util/ipc/IpcEndpointBindException.java      |    47 +
 .../internal/util/ipc/IpcEndpointFactory.java   |    84 +
 .../internal/util/ipc/IpcEndpointType.java      |    29 +
 .../internal/util/ipc/IpcServerEndpoint.java    |    73 +
 .../util/ipc/IpcServerEndpointDeserializer.java |    66 +
 .../internal/util/ipc/IpcToNioAdapter.java      |   250 +
 .../ipc/loopback/GridIpcClientTcpEndpoint.java  |    87 -
 .../ipc/loopback/GridIpcServerTcpEndpoint.java  |   179 -
 .../util/ipc/loopback/IpcClientTcpEndpoint.java |    87 +
 .../util/ipc/loopback/IpcServerTcpEndpoint.java |   179 +
 .../GridIpcOutOfSystemResourcesException.java   |    59 -
 .../GridIpcSharedMemoryClientEndpoint.java      |   336 -
 .../shmem/GridIpcSharedMemoryInitRequest.java   |    67 -
 .../shmem/GridIpcSharedMemoryInitResponse.java  |   171 -
 .../shmem/GridIpcSharedMemoryInputStream.java   |    99 -
 .../shmem/GridIpcSharedMemoryNativeLoader.java  |   242 -
 ...cSharedMemoryOperationTimedoutException.java |    59 -
 .../shmem/GridIpcSharedMemoryOutputStream.java  |    80 -
 .../GridIpcSharedMemoryServerEndpoint.java      |   707 -
 .../ipc/shmem/GridIpcSharedMemorySpace.java     |   374 -
 .../ipc/shmem/GridIpcSharedMemoryUtils.java     |   242 -
 .../shmem/IpcOutOfSystemResourcesException.java |    59 +
 .../shmem/IpcSharedMemoryClientEndpoint.java    |   336 +
 .../ipc/shmem/IpcSharedMemoryInitRequest.java   |    67 +
 .../ipc/shmem/IpcSharedMemoryInitResponse.java  |   171 +
 .../ipc/shmem/IpcSharedMemoryInputStream.java   |    99 +
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  |   261 +
 ...cSharedMemoryOperationTimedoutException.java |    59 +
 .../ipc/shmem/IpcSharedMemoryOutputStream.java  |    80 +
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   707 +
 .../util/ipc/shmem/IpcSharedMemorySpace.java    |   374 +
 .../util/ipc/shmem/IpcSharedMemoryUtils.java    |   242 +
 .../ignite/internal/util/lang/GridFunc.java     |    53 +-
 .../internal/util/nio/GridBufferedParser.java   |     2 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |    18 +-
 .../ignite/internal/util/nio/GridNioFuture.java |     2 +-
 .../util/nio/GridShmemCommunicationClient.java  |     4 +-
 .../util/nodestart/GridNodeStartUtils.java      |    10 +-
 .../util/offheap/unsafe/GridUnsafeMemory.java   |     2 +-
 .../util/portscanner/GridJmxPortFinder.java     |     4 +-
 .../apache/ignite/internal/util/typedef/X.java  |    10 +-
 .../internal/util/typedef/internal/U.java       |     4 +-
 .../apache/ignite/internal/visor/VisorJob.java  |     2 +-
 .../internal/visor/VisorMultiNodeTask.java      |     2 +-
 .../ignite/internal/visor/cache/VisorCache.java |     2 +-
 .../visor/cache/VisorCachePreloadTask.java      |     6 +-
 .../compute/VisorComputeMonitoringHolder.java   |    10 +-
 .../visor/misc/VisorResolveHostNameTask.java    |     2 +-
 .../visor/node/VisorBasicConfiguration.java     |    28 +-
 .../visor/node/VisorEmailConfiguration.java     |    14 +-
 .../visor/node/VisorGridConfiguration.java      |     2 +-
 .../visor/node/VisorLifecycleConfiguration.java |     2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |     6 +-
 .../visor/node/VisorRestConfiguration.java      |     4 +-
 .../internal/visor/query/VisorQueryTask.java    |     2 +-
 .../internal/visor/query/VisorQueryUtils.java   |     4 +-
 .../apache/ignite/jdbc/IgniteJdbcDriver.java    |     2 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |     4 +-
 .../ignite/lang/IgniteAsyncSupportAdapter.java  |    38 +-
 .../org/apache/ignite/lang/IgniteFuture.java    |   189 -
 .../apache/ignite/lifecycle/LifecycleBean.java  |     4 +-
 .../ignite/logger/java/IgniteJavaLogger.java    |     4 +-
 .../java/IgniteJavaLoggerFileHandler.java       |     4 +-
 .../optimized/IgniteOptimizedMarshaller.java    |     4 +-
 .../segmentation/GridSegmentationPolicy.java    |     2 +-
 .../ignite/scheduler/SchedulerFuture.java       |     4 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |     4 +-
 .../ignite/spi/IgniteSpiManagementMBean.java    |     2 +-
 .../ignite/spi/checkpoint/CheckpointSpi.java    |     2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |     6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    36 +-
 .../ignite/spi/deployment/DeploymentSpi.java    |     2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |     8 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |     2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |    18 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |     2 +-
 .../tcp/ipfinder/vm/TcpDiscoveryVmIpFinder.java |     2 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |     4 +-
 .../ignite/startup/BasicWarmupClosure.java      |     6 +-
 .../startup/cmdline/CommandLineStartup.java     |    18 +-
 .../startup/cmdline/CommandLineTransformer.java |     8 +-
 .../StreamerCacheAffinityEventRouter.java       |     2 +-
 .../ignite/thread/IgniteThreadFactory.java      |     2 +-
 .../apache/ignite/transactions/IgniteTx.java    |     2 +-
 .../core/src/test/bin/start-nodes-custom.bat    |     2 +-
 modules/core/src/test/bin/start-nodes-custom.sh |     2 +-
 .../spring-cache-client-benchmark-1.xml         |     2 +-
 .../spring-cache-client-benchmark-2.xml         |     2 +-
 .../spring-cache-client-benchmark-3.xml         |     2 +-
 modules/core/src/test/config/example-cache.xml  |     2 +-
 modules/core/src/test/config/ggfs-loopback.xml  |     2 +-
 modules/core/src/test/config/ggfs-shmem.xml     |     2 +-
 modules/core/src/test/config/log4j-test.xml     |     2 +-
 .../core/src/test/config/spring-cache-load.xml  |     2 +-
 .../core/src/test/config/spring-multicache.xml  |     2 +-
 .../average/spring-streamer-average-base.xml    |     2 +-
 .../average/spring-streamer-average-local.xml   |     2 +-
 .../average/spring-streamer-average-random.xml  |     2 +-
 .../config/streamer/spring-streamer-base.xml    |     2 +-
 modules/core/src/test/config/tests.properties   |    10 +-
 .../ignite/GridExceptionHelpLinksSelfTest.java  |   106 -
 .../ignite/GridExternalizableAbstractTest.java  |    41 -
 .../org/apache/ignite/GridTestTaskSession.java  |     4 +-
 .../IgniteExceptionHelpLinksSelfTest.java       |   105 +
 .../IgniteExternalizableAbstractTest.java       |    41 +
 ...CachePartitionFairAffinityNodesSelfTest.java |     6 +-
 .../store/GridCacheBalancingStoreSelfTest.java  |     3 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |     7 +-
 .../fs/GridGgfsEventsAbstractSelfTest.java      |     2 +-
 .../ignite/fs/GridGgfsFragmentizerSelfTest.java |     2 +-
 .../internal/GridDiscoveryEventSelfTest.java    |     3 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |     4 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |     2 +-
 .../internal/GridExecutorServiceTest.java       |   315 -
 .../GridExplicitImplicitDeploymentSelfTest.java |   476 -
 .../internal/GridFactoryVmShutdownTest.java     |     2 +-
 .../ignite/internal/GridHomePathSelfTest.java   |     2 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |    56 +-
 .../ignite/internal/GridKernalTestUtils.java    |     3 +-
 .../internal/GridListenActorSelfTest.java       |     3 +-
 .../internal/GridMultipleJobsSelfTest.java      |     6 +-
 .../GridNodeVisorAttributesSelfTest.java        |     4 +-
 .../internal/GridProjectionAbstractTest.java    |    20 +-
 .../ignite/internal/GridReduceSelfTest.java     |     2 +-
 .../ignite/internal/GridStartStopSelfTest.java  |     2 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |     5 +-
 .../internal/GridTaskListenerSelfTest.java      |     4 +-
 .../GridTopicExternalizableSelfTest.java        |     3 +-
 .../GridTopologyBuildVersionSelfTest.java       |     2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |     3 +-
 .../ignite/internal/GridVersionSelfTest.java    |     8 +-
 .../internal/IgniteExecutorServiceTest.java     |   315 +
 ...gniteExplicitImplicitDeploymentSelfTest.java |   476 +
 .../GridCheckpointManagerAbstractSelfTest.java  |     4 +-
 .../GridCommunicationSendMessageSelfTest.java   |     4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |     2 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |     6 +-
 .../events/GridEventStorageManagerSelfTest.java |     3 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |     2 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |     6 +-
 .../GridCacheAbstractFailoverSelfTest.java      |     3 +-
 .../cache/GridCacheAbstractFlagsTest.java       |     4 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |     5 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   381 +-
 .../GridCacheAbstractIteratorsSelfTest.java     |    10 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |    28 +-
 .../GridCacheAbstractProjectionSelfTest.java    |    50 +
 .../GridCacheAbstractRemoveFailureTest.java     |    12 +-
 .../cache/GridCacheAbstractSelfTest.java        |     2 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |     4 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |     8 +-
 .../cache/GridCacheConcurrentMapTest.java       |     6 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |     4 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |     4 +-
 .../GridCacheFinishPartitionsSelfTest.java      |    27 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |     6 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |     4 +-
 .../GridCacheGroupLockAbstractSelfTest.java     |     6 +-
 .../GridCacheGroupLockFailoverSelfTest.java     |     5 +-
 .../cache/GridCacheIncrementTransformTest.java  |     4 +-
 .../processors/cache/GridCacheLeakTest.java     |     2 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |    18 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |     9 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |     6 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |     5 +-
 .../cache/GridCacheMvccFlagsTest.java           |     4 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |     4 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |     4 +-
 .../processors/cache/GridCacheMvccSelfTest.java |     4 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |     2 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |     2 +-
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |     5 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |     2 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |     6 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |     2 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |     8 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    17 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |     2 +-
 .../GridCacheReferenceCleanupSelfTest.java      |     6 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |     4 +-
 .../GridCacheReturnValueTransferSelfTest.java   |     2 +-
 .../cache/GridCacheSlowTxWarnTest.java          |     6 +-
 .../processors/cache/GridCacheStopSelfTest.java |     6 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |     4 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |     4 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |     2 +-
 .../cache/GridCacheTtlManagerLoadTest.java      |     5 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |     2 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |     6 +-
 .../GridCacheVariableTopologySelfTest.java      |     4 +-
 ...idCacheWriteBehindStoreAbstractSelfTest.java |     4 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |     4 +-
 .../GridCacheWriteBehindStoreSelfTest.java      |     6 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |     7 +-
 .../cache/IgniteCacheInvokeAbstractTest.java    |    10 +-
 .../processors/cache/IgniteTxAbstractTest.java  |     2 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |     3 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |     2 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |     6 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |     6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |     2 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |    27 +-
 ...actQueueFailoverDataConsistencySelfTest.java |     9 +-
 .../GridCacheCountDownLatchSelfTest.java        |     8 +-
 .../GridCacheMultiNodeDataStructureTest.java    |     4 +-
 .../GridCacheQueueCleanupSelfTest.java          |     5 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |     7 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |    17 +-
 .../GridCacheSetAbstractSelfTest.java           |    15 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |     6 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |     6 +-
 ...dCachePartitionedQueueEntryMoveSelfTest.java |     6 +-
 .../GridCacheAbstractJobExecutionTest.java      |     6 +-
 .../GridCacheAtomicTimeoutSelfTest.java         |    11 +-
 .../GridCacheBasicOpAbstractTest.java           |     7 +-
 .../distributed/GridCacheEventAbstractTest.java |     2 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |     2 +-
 .../distributed/GridCacheLockAbstractTest.java  |     6 +-
 .../GridCacheMultiNodeAbstractTest.java         |    15 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |     2 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |     4 +-
 .../GridCacheNodeFailureAbstractTest.java       |     8 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |   288 +
 ...iteTxConsistencyRestartAbstractSelfTest.java |     4 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |    10 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |    22 +-
 .../IgniteTxPreloadAbstractTest.java            |     4 +-
 .../dht/GridCacheAtomicFullApiSelfTest.java     |    11 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |     2 +-
 .../dht/GridCacheColocatedDebugTest.java        |    11 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |     2 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    10 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |    12 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |     2 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |     2 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |     6 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |     5 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |     5 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |    60 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |     8 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |     5 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |     4 +-
 ...GridCacheValueConsistencyAtomicSelfTest.java |     2 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |     6 +-
 .../near/GridCacheNearReadersSelfTest.java      |     4 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |     2 +-
 .../GridCachePartitionedEntryLockSelfTest.java  |     5 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |     6 +-
 .../GridCachePartitionedFullApiSelfTest.java    |     4 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |     2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |     5 +-
 .../GridCachePartitionedProjectionSelfTest.java |     4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |     6 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |    11 +-
 .../GridCacheReplicatedEvictionSelfTest.java    |     5 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |     5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |     4 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |     4 +-
 .../cache/eviction/GridCacheMockEntry.java      |    27 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |     4 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |     2 +-
 .../IgniteCacheTxStoreSessionTest.java          |    11 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |     4 +-
 .../clock/GridTimeSyncProcessorSelfTest.java    |     6 +-
 .../closure/GridClosureProcessorSelfTest.java   |    27 +-
 .../continuous/GridEventConsumeSelfTest.java    |    10 +-
 .../GridDataLoaderProcessorSelfTest.java        |    23 +-
 .../processors/fs/GridGgfsAbstractSelfTest.java |     6 +-
 .../fs/GridGgfsDataManagerSelfTest.java         |     9 +-
 .../processors/fs/GridGgfsModesSelfTest.java    |     4 +-
 .../fs/GridGgfsProcessorSelfTest.java           |     2 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |     6 +-
 ...pcEndpointRegistrationOnWindowsSelfTest.java |     4 +-
 .../processors/fs/GridGgfsSizeSelfTest.java     |     8 +-
 .../processors/fs/GridGgfsTaskSelfTest.java     |     5 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |    13 +-
 .../GridServiceProcessorAbstractSelfTest.java   |    20 +-
 .../GridServiceProcessorMultiNodeSelfTest.java  |     8 +-
 .../GridServiceReassignmentSelfTest.java        |     2 +-
 .../streamer/GridStreamerFailoverSelfTest.java  |     4 +-
 .../timeout/GridTimeoutProcessorSelfTest.java   |     5 +-
 ...artupWithSpecifiedWorkDirectorySelfTest.java |     4 +-
 ...tartupWithUndefinedGridGainHomeSelfTest.java |    10 +-
 .../internal/util/GridTestClockTimer.java       |     6 +-
 .../ignite/internal/util/GridUtilsSelfTest.java |   717 -
 .../internal/util/IgniteUtilsSelfTest.java      |   728 +
 .../util/future/GridCompoundFutureSelfTest.java |     8 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |     7 +-
 .../util/future/GridFinishedFutureSelfTest.java |     9 +-
 .../util/future/GridFutureAdapterSelfTest.java  |    23 +-
 .../future/GridFutureListenPerformanceTest.java |     5 +-
 ...GridUnsafeDataOutputArraySizingSelfTest.java |     2 +-
 ...idIpcServerEndpointDeserializerSelfTest.java |   160 -
 .../IpcServerEndpointDeserializerSelfTest.java  |   160 +
 .../ipc/shmem/GgfsSharedMemoryTestClient.java   |    76 +
 .../ipc/shmem/GgfsSharedMemoryTestServer.java   |    71 +
 .../shmem/GridGgfsSharedMemoryTestClient.java   |    76 -
 .../shmem/GridGgfsSharedMemoryTestServer.java   |    71 -
 ...idIpcSharedMemoryCrashDetectionSelfTest.java |   500 -
 .../shmem/GridIpcSharedMemoryFakeClient.java    |    36 -
 .../shmem/GridIpcSharedMemoryNodeStartup.java   |    87 -
 .../shmem/GridIpcSharedMemorySpaceSelfTest.java |   259 -
 .../shmem/GridIpcSharedMemoryUtilsSelfTest.java |    84 -
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |   500 +
 .../ipc/shmem/IpcSharedMemoryFakeClient.java    |    36 +
 .../IpcSharedMemoryNativeLoaderSelfTest.java    |    78 +
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |    87 +
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |   267 +
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |    84 +
 .../LoadWithCorruptedLibFileTestRunner.java     |    65 +
 .../GridIpcSharedMemoryBenchmarkParty.java      |    35 -
 .../GridIpcSharedMemoryBenchmarkReader.java     |   133 -
 .../GridIpcSharedMemoryBenchmarkWriter.java     |   125 -
 .../IpcSharedMemoryBenchmarkParty.java          |    35 +
 .../IpcSharedMemoryBenchmarkReader.java         |   133 +
 .../IpcSharedMemoryBenchmarkWriter.java         |   125 +
 .../internal/util/nio/GridNioSelfTest.java      |     8 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |     7 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |     3 +-
 ...apPartitionedMapPerformanceAbstractTest.java |    10 +-
 .../unsafe/GridUnsafeMemorySelfTest.java        |     6 +-
 .../ignite/jvmtest/ConcurrentMapTest.java       |     4 +-
 .../ignite/jvmtest/NetworkFailureTest.java      |    16 +-
 .../QueueSizeCounterMultiThreadedTest.java      |     4 +-
 .../jvmtest/ReadWriteLockMultiThreadedTest.java |    10 +-
 .../ignite/lang/GridBasicPerformanceTest.java   |     5 +-
 .../apache/ignite/lang/GridFuncSelfTest.java    |    13 +-
 .../lang/GridFutureListenPerformanceTest.java   |     5 +-
 .../ignite/lang/GridSetWrapperSelfTest.java     |     2 +-
 .../loadtest/GridSingleExecutionTest.java       |     4 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |     6 +-
 .../cache/GridCacheAbstractLoadTest.java        |     8 +-
 .../cache/GridCacheDataStructuresLoadTest.java  |     2 +-
 .../loadtests/cache/GridCacheLoadTest.java      |     2 +-
 .../cache/GridCachePutRemoveLoadTest.java       |     2 +-
 .../loadtests/cache/GridCacheSwapLoadTest.java  |    13 +-
 .../GridCacheWriteBehindStoreLoadTest.java      |     4 +-
 .../capacity/spring-capacity-cache.xml          |     2 +-
 .../loadtests/colocation/GridTestMain.java      |    10 +-
 .../loadtests/colocation/spring-colocation.xml  |     2 +-
 .../communication/GridIoManagerBenchmark.java   |    10 +-
 .../communication/GridIoManagerBenchmark0.java  |    22 +-
 .../GridContinuousOperationsLoadTest.java       |    22 +-
 .../ignite/loadtests/dsi/GridDsiPerfJob.java    |     2 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |     5 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |     7 +-
 .../loadtests/mapper/GridNodeStartup.java       |     4 +-
 .../mergesort/GridMergeSortLoadTask.java        |     6 +-
 .../loadtests/nio/GridNioBenchmarkTest.java     |     2 +-
 ...apPartitionedMapPerformanceAbstractTest.java |    10 +-
 .../streamer/GridStreamerIndexLoadTest.java     |    10 +-
 .../ignite/loadtests/util/GridLoadTestArgs.java |     8 +-
 .../marshaller/GridMarshallerAbstractTest.java  |    20 +-
 .../ignite/messaging/GridMessagingSelfTest.java |     8 +-
 ...redFsCheckpointSpiMultiThreadedSelfTest.java |     8 +-
 .../collision/GridTestCollisionTaskSession.java |     3 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |     3 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |     3 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |     9 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |     7 +-
 .../tcp/GridTcpDiscoveryMultiThreadedTest.java  |     7 +-
 .../discovery/tcp/GridTcpDiscoverySelfTest.java |    14 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |     7 +-
 .../startup/GridRandomCommandLineLoader.java    |    10 +-
 .../ignite/startup/GridVmNodesStarter.java      |     8 +-
 .../GridCommandLineTransformerSelfTest.java     |     6 +-
 .../GridCacheStoreValueBytesTest.java           |     7 +-
 .../index/GridStreamerIndexSelfTest.java        |     8 +-
 .../window/GridStreamerWindowSelfTest.java      |     4 +-
 .../ignite/testframework/GridTestUtils.java     |    22 +-
 .../config/GridTestProperties.java              |    16 +-
 .../testframework/junits/GridAbstractTest.java  |    26 +-
 .../junits/GridTestKernalContext.java           |     2 +-
 .../cache/GridAbstractCacheStoreSelfTest.java   |     3 +-
 .../junits/common/GridCommonAbstractTest.java   |    26 +-
 .../logger/GridLog4jRollingFileAppender.java    |     4 +-
 .../junits/logger/GridTestLog4jLogger.java      |     8 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |     2 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |     1 +
 .../testsuites/IgniteComputeGridTestSuite.java  |     4 +-
 .../ignite/testsuites/IgniteFsTestSuite.java    |     2 +-
 .../IgniteIpcSharedMemorySelfTestSuite.java     |     7 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |     2 +-
 ...dConcurrentLinkedDequeMultiThreadedTest.java |     6 +-
 ...rrentLinkedHashMapMultiThreadedSelfTest.java |    10 +-
 .../apache/ignite/util/GridIndexFillTest.java   |     4 +-
 .../ignite/util/GridSnapshotLockSelfTest.java   |     6 +-
 .../util/GridSpinReadWriteLockSelfTest.java     |     4 +-
 .../src/test/resources/log/gridgain.log.tst     | 13706 -----------------
 .../core/src/test/resources/log/ignite.log.tst  | 13706 +++++++++++++++++
 modules/core/src/test/resources/readme.txt      |     2 +-
 .../core/src/test/webapp/META-INF/gg-config.xml |     2 +-
 modules/core/src/test/webapp/WEB-INF/web.xml    |     2 +-
 modules/email/readme.txt                        |    14 +-
 .../processors/email/IgniteEmailProcessor.java  |     5 +-
 modules/hadoop/readme.txt                       |    14 +-
 .../GridHadoopClientProtocolProvider.java       |     8 +-
 .../fs/hadoop/v1/GridGgfsHadoopFileSystem.java  |     6 +-
 .../fs/hadoop/v2/GridGgfsHadoopFileSystem.java  |     6 +-
 .../apache/ignite/hadoop/GridHadoopSetup.java   |    16 +-
 .../internal/fs/hadoop/GridGgfsHadoopIpcIo.java |    10 +-
 .../hadoop/GridHadoopClassLoader.java           |     4 +-
 .../processors/hadoop/GridHadoopImpl.java       |     6 +-
 .../hadoop/IgniteHadoopProcessor.java           |     5 +-
 .../hadoop/jobtracker/GridHadoopJobTracker.java |    19 +-
 .../GridHadoopDefaultMapReducePlanner.java      |     2 +-
 .../proto/GridHadoopProtocolJobStatusTask.java  |     7 +-
 .../hadoop/shuffle/GridHadoopShuffle.java       |     2 +-
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |     7 +-
 .../GridHadoopExternalTaskExecutor.java         |    18 +-
 .../child/GridHadoopChildProcessRunner.java     |    18 +-
 .../child/GridHadoopExternalProcessStarter.java |     2 +-
 .../GridHadoopExternalCommunication.java        |    26 +-
 .../GridHadoopIpcToNioAdapter.java              |     6 +-
 ...doop20FileSystemLoopbackPrimarySelfTest.java |     2 +-
 ...sHadoop20FileSystemShmemPrimarySelfTest.java |     2 +-
 .../GridGgfsHadoopFileSystemClientSelfTest.java |     4 +-
 ...idGgfsHadoopFileSystemHandshakeSelfTest.java |     2 +-
 ...ridGgfsHadoopFileSystemIpcCacheSelfTest.java |     2 +-
 ...adoopFileSystemLoopbackAbstractSelfTest.java |     2 +-
 ...fsHadoopFileSystemShmemAbstractSelfTest.java |     8 +-
 .../fs/GridGgfsNearOnlyMultiNodeSelfTest.java   |     4 +-
 .../ignite/fs/IgniteFsEventsTestSuite.java      |     4 +-
 .../hadoop/GridHadoopPopularWordsTest.java      |     6 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |    12 +-
 .../hadoop/GridHadoopJobTrackerSelfTest.java    |     5 +-
 .../GridHadoopMapReduceEmbeddedSelfTest.java    |     4 +-
 .../hadoop/GridHadoopMapReduceTest.java         |     4 +-
 .../hadoop/GridHadoopTaskExecutionSelfTest.java |    12 +-
 .../GridHadoopExecutorServiceTest.java          |     4 +-
 ...GridHadoopExternalTaskExecutionSelfTest.java |     6 +-
 .../ggfs/GridGgfsPerformanceBenchmark.java      |     4 +-
 modules/hibernate/readme.txt                    |    16 +-
 .../hibernate/CacheHibernateBlobStore.java      |   593 +
 .../CacheHibernateBlobStoreEntry.hbm.xml        |    29 +
 .../hibernate/CacheHibernateBlobStoreEntry.java |    86 +
 .../hibernate/GridCacheHibernateBlobStore.java  |   593 -
 .../GridCacheHibernateBlobStoreEntry.hbm.xml    |    29 -
 .../GridCacheHibernateBlobStoreEntry.java       |    86 -
 ...idHibernateL2CacheTransactionalSelfTest.java |     2 +-
 .../CacheHibernateBlobStoreSelfTest.java        |   108 +
 .../GridCacheHibernateBlobStoreSelfTest.java    |   108 -
 .../cache/store/hibernate/hibernate.cfg.xml     |     2 +-
 .../testsuites/IgniteHibernateTestSuite.java    |     2 +-
 modules/indexing/readme.txt                     |    14 +-
 .../processors/query/h2/IgniteH2Indexing.java   |    12 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |     2 +-
 .../cache/GridCacheAbstractQuerySelfTest.java   |     8 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |     8 +-
 .../cache/GridCacheQueryLoadSelfTest.java       |     2 +-
 .../GridCacheQueryMultiThreadedSelfTest.java    |    26 +-
 ...idCacheReduceQueryMultithreadedSelfTest.java |     6 +-
 .../near/GridCacheQueryNodeRestartSelfTest.java |     5 +-
 .../GridCacheReplicatedFieldsQuerySelfTest.java |     2 +-
 .../GridCacheReplicatedQuerySelfTest.java       |     6 +-
 .../query/h2/GridH2IndexRebuildTest.java        |     7 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |     6 +-
 .../query/h2/sql/GridQueryParsingTest.java      |     4 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |     4 +-
 modules/jcl/readme.txt                          |    14 +-
 .../apache/ignite/logger/jcl/GridJclLogger.java |     4 +-
 modules/jta/readme.txt                          |    14 +-
 .../apache/ignite/cache/jta/CacheTmLookup.java  |    50 +
 .../ignite/cache/jta/GridCacheTmLookup.java     |    50 -
 .../cache/jta/jndi/CacheJndiTmLookup.java       |    74 +
 .../cache/jta/jndi/GridCacheJndiTmLookup.java   |    74 -
 .../jta/reflect/CacheReflectionTmLookup.java    |   115 +
 .../reflect/GridCacheReflectionTmLookup.java    |   115 -
 .../processors/cache/jta/CacheJtaManager.java   |     4 +-
 ...CacheJtaConfigurationValidationSelfTest.java |     2 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |     2 +-
 .../GridTmLookupLifecycleAwareSelfTest.java     |     6 +-
 modules/log4j/readme.txt                        |    14 +-
 .../ignite/logger/log4j/IgniteLog4jLogger.java  |     8 +-
 .../logger/log4j/IgniteLog4jNodeIdFilePath.java |     8 +-
 .../log4j/GridLog4jCorrectFileNameTest.java     |     4 +-
 modules/rest-http/readme.txt                    |    14 +-
 .../http/jetty/GridJettyRestProtocol.java       |    16 +-
 .../rest/protocols/http/jetty/rest.html         |     6 +-
 modules/scalar/pom.xml                          |     6 +-
 modules/scalar/readme.txt                       |    10 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |    25 +-
 modules/schedule/readme.txt                     |    14 +-
 .../processors/schedule/ScheduleFutureImpl.java |    26 +-
 .../schedule/GridScheduleSelfTest.java          |    13 +-
 modules/slf4j/readme.txt                        |    14 +-
 modules/spring/readme.txt                       |    14 +-
 .../java/org/apache/ignite/IgniteSpring.java    |    10 +-
 .../ignite/cache/spring/SpringCacheManager.java |     6 +-
 .../cache/spring/SpringDynamicCacheManager.java |     2 +-
 .../spring/IgniteSpringProcessorImpl.java       |    10 +-
 .../ignite/internal/GridFactorySelfTest.java    |     2 +-
 modules/ssh/readme.txt                          |    14 +-
 .../util/nodestart/GridNodeCallableImpl.java    |    20 +-
 .../internal/GridNodeStartUtilsSelfTest.java    |     6 +-
 .../GridProjectionStartStopRestartSelfTest.java |     4 +-
 .../ignite/tools/javadoc/GridLinkTaglet.java    |   165 -
 .../ignite/tools/javadoc/IgniteLinkTaglet.java  |   165 +
 .../optimized/OptimizedClassNamesGenerator.java |    18 +-
 modules/urideploy/readme.txt                    |    14 +-
 .../uri/GridUriDeploymentFileProcessor.java     |     2 +-
 .../deployment/uri/GridUriDeploymentSpi.java    |  1367 --
 .../uri/GridUriDeploymentSpiMBean.java          |    53 -
 .../spi/deployment/uri/UriDeploymentSpi.java    |  1367 ++
 .../deployment/uri/UriDeploymentSpiMBean.java   |    53 +
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |     4 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |     2 +-
 .../uri/GridUriDeploymentAbstractSelfTest.java  |     2 +-
 .../GridUriDeploymentClassLoaderSelfTest.java   |     2 +-
 ...riDeploymentClassloaderRegisterSelfTest.java |     4 +-
 .../uri/GridUriDeploymentConfigSelfTest.java    |    10 +-
 .../GridUriDeploymentFileProcessorSelfTest.java |     2 +-
 .../uri/GridUriDeploymentMd5CheckSelfTest.java  |     2 +-
 ...loymentMultiScannersErrorThrottlingTest.java |     2 +-
 .../GridUriDeploymentMultiScannersSelfTest.java |     2 +-
 .../uri/GridUriDeploymentSimpleSelfTest.java    |    10 +-
 .../file/GridFileDeploymentSelfTest.java        |     6 +-
 .../GridFileDeploymentUndeploySelfTest.java     |     8 +-
 .../uri/scanners/ftp/GridFtpDeploymentTest.java |     2 +-
 .../http/GridHttpDeploymentSelfTest.java        |     2 +-
 .../IgniteUriDeploymentTestSuite.java           |     3 +-
 .../scala/org/apache/ignite/visor/Packet.scala  |     2 +-
 .../ignite/visor/commands/VisorConsole.scala    |     6 +-
 .../commands/alert/VisorAlertCommand.scala      |     2 +-
 .../config/VisorConfigurationCommand.scala      |     6 +-
 .../ignite/visor/commands/deploy/Packet.scala   |     4 +-
 .../commands/deploy/VisorDeployCommand.scala    |    22 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |     4 +-
 .../commands/events/VisorEventsCommand.scala    |     4 +-
 .../visor/commands/gc/VisorGcCommand.scala      |    13 +-
 .../visor/commands/node/VisorNodeCommand.scala  |     4 +-
 .../ignite/visor/commands/start/Packet.scala    |     6 +-
 .../commands/start/VisorStartCommand.scala      |    16 +-
 .../commands/tasks/VisorTasksCommand.scala      |     4 +-
 .../commands/top/VisorTopologyCommand.scala     |     6 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |     8 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    41 +-
 .../visor/commands/gc/VisorGcCommandSpec.scala  |    25 +-
 .../testsuites/VisorConsoleSelfTestSuite.scala  |     2 +-
 modules/web/readme.txt                          |    18 +-
 .../cache/websession/GridWebSessionFilter.java  |    26 +-
 .../GridServletContextListenerStartup.java      |   181 -
 .../startup/servlet/GridServletStartup.java     |   187 -
 .../IgniteServletContextListenerStartup.java    |   181 +
 .../startup/servlet/IgniteServletStartup.java   |   187 +
 .../internal/websession/WebSessionSelfTest.java |    12 +-
 .../websession/WebSessionServerStart.java       |     2 +-
 pom.xml                                         |    26 +-
 948 files changed, 46464 insertions(+), 45536 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f49e955/pom.xml
----------------------------------------------------------------------


[02/12] incubator-ignite git commit: Merge branches 'ignite-102' and 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102

Posted by ak...@apache.org.
Merge branches 'ignite-102' and 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102


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

Branch: refs/heads/sprint-1
Commit: 99e3fa618bcb4581626188046933b157a2acc577
Parents: b0031cc c224017
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jan 28 10:24:53 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jan 28 10:24:53 2015 +0700

----------------------------------------------------------------------
 NOTICE.txt                                      |   14 +-
 examples/config/example-cache.xml               |    9 +-
 examples/config/example-compute.xml             |    9 +-
 examples/config/example-streamer.xml            |    9 +-
 examples/config/filesystem/example-ggfs.xml     |    9 +-
 .../examples/misc/springbean/spring-bean.xml    |    9 +-
 .../examples/ScalarStartStopExample.scala       |   73 -
 .../examples/IgniteFsExamplesSelfTest.java      |   26 +-
 .../tests/examples/ScalarExamplesSelfTest.scala |    5 -
 .../spi/checkpoint/s3/S3CheckpointSpiMBean.java |   12 +-
 .../integration/ClientAbstractSelfTest.java     |   20 +-
 .../rest/AbstractRestProcessorSelfTest.java     |    2 +
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../rest/RestBinaryProtocolSelfTest.java        |    9 +-
 .../rest/RestMemcacheProtocolSelfTest.java      |    9 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   10 +
 .../java/org/apache/ignite/IgniteCache.java     |   15 +
 .../apache/ignite/cache/CacheEntryEvent.java    |    2 +-
 .../org/apache/ignite/cache/CacheMBean.java     |  295 ----
 .../org/apache/ignite/cache/CacheManager.java   |  164 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  416 +++++-
 .../java/org/apache/ignite/cache/GridCache.java |   15 +-
 .../ignite/cache/affinity/CacheAffinity.java    |    9 +-
 .../fifo/CacheFifoEvictionPolicyMBean.java      |   10 +-
 ...CacheGgfsPerBlockLruEvictionPolicyMBean.java |   18 +-
 .../lru/CacheLruEvictionPolicyMBean.java        |   10 +-
 .../random/CacheRandomEvictionPolicyMBean.java  |    6 +-
 .../ignite/cache/store/CacheStoreSession.java   |    2 +-
 .../client/router/GridTcpRouterMBean.java       |   16 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |  697 +++++----
 .../org/apache/ignite/cluster/ClusterNode.java  |   40 +-
 .../ignite/cluster/ClusterNodeMetrics.java      |  695 ---------
 .../configuration/IgniteConfiguration.java      |   27 +-
 .../ignite/dotnet/GridDotNetConfiguration.java  |  113 --
 .../dotnet/GridDotNetPortableConfiguration.java |  207 ---
 .../GridDotNetPortableTypeConfiguration.java    |  219 ---
 .../java/org/apache/ignite/dotnet/package.html  |   23 -
 .../ignite/internal/ClusterGroupAdapter.java    |    2 +-
 .../internal/ClusterLocalNodeMetrics.java       |  311 ----
 .../ClusterLocalNodeMetricsMXBeanImpl.java      |  309 ++++
 .../ignite/internal/ClusterMetricsImpl.java     |  813 ----------
 .../ignite/internal/ClusterMetricsSnapshot.java | 1397 ++++++++++++++++++
 .../internal/ClusterNodeMetricsMBean.java       |   29 -
 .../org/apache/ignite/internal/GridGainEx.java  |    8 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   95 +-
 .../internal/GridJobExecuteRequestV2.java       |  185 ---
 .../org/apache/ignite/internal/GridKernal.java  |   32 +-
 .../org/apache/ignite/internal/GridTopic.java   |    5 +-
 .../ignite/internal/IgniteClusterAsyncImpl.java |    1 -
 .../internal/IgniteThreadPoolMBeanAdapter.java  |  149 --
 .../internal/IgniteThreadPoolMXBeanAdapter.java |  129 ++
 .../ignite/internal/IgnitionMBeanAdapter.java   |   59 -
 .../ignite/internal/IgnitionMXBeanAdapter.java  |   59 +
 .../discovery/GridDiscoveryManager.java         |    4 +-
 .../internal/mxbean/IgniteStandardMXBean.java   |  277 ++++
 .../apache/ignite/internal/mxbean/package.html  |   24 +
 .../affinity/GridAffinityAssignment.java        |   10 +-
 .../affinity/GridAffinityProcessor.java         |  330 ++++-
 .../processors/cache/CacheMetricsImpl.java      |  586 ++++++++
 .../cache/CacheMetricsMXBeanImpl.java           |  310 ++++
 .../processors/cache/CacheMetricsSnapshot.java  |  518 +++++++
 .../processors/cache/GridCacheAdapter.java      |  377 ++++-
 .../cache/GridCacheEvictionManager.java         |    3 +
 .../processors/cache/GridCacheMBeanAdapter.java |  217 ---
 .../processors/cache/GridCacheMapEntry.java     |   55 +-
 .../cache/GridCacheMetricsAdapter.java          |  260 ----
 .../processors/cache/GridCacheProcessor.java    |   37 -
 .../processors/cache/GridCacheProxyImpl.java    |   25 +-
 .../processors/cache/GridCacheTtlManager.java   |    3 +
 .../processors/cache/GridCacheVersionEx.java    |    9 +
 .../processors/cache/IgniteCacheProxy.java      |   28 +-
 .../cache/distributed/dht/GridDhtCache.java     |   12 +-
 .../dht/GridPartitionedGetFuture.java           |    2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   30 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    4 +-
 .../distributed/near/GridNearCacheAdapter.java  |    7 -
 .../distributed/near/GridNearCacheEntry.java    |    3 +-
 .../distributed/near/GridNearGetFuture.java     |    8 +-
 .../distributed/near/GridNearLockFuture.java    |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |   24 +-
 .../cache/query/GridCacheQueriesEx.java         |   16 +
 .../cache/query/GridCacheQueriesImpl.java       |   15 +
 .../cache/query/GridCacheQueriesProxy.java      |   24 +
 .../cache/query/GridCacheQueryManager.java      |   64 +-
 .../cache/query/GridCacheSqlQuery.java          |   99 ++
 .../cache/query/GridCacheSqlResult.java         |   21 +
 .../cache/query/GridCacheTwoStepQuery.java      |   74 +
 .../GridCacheContinuousQueryManager.java        |   11 -
 .../cache/transactions/IgniteTxManager.java     |    5 +-
 .../closure/GridClosureProcessor.java           |  624 ++++++--
 .../GridMasterLeaveAwareComputeJobAdapter.java  |   36 -
 .../processors/query/GridQueryIndexing.java     |   19 +
 .../processors/query/GridQueryProcessor.java    |   36 +
 .../processors/resource/GridResourceUtils.java  |    4 +-
 .../handlers/cache/GridCacheCommandHandler.java |    7 +-
 .../handlers/cache/GridCacheRestMetrics.java    |   77 +-
 .../top/GridTopologyCommandHandler.java         |    2 +-
 .../processors/task/GridTaskWorker.java         |   52 +-
 .../apache/ignite/internal/util/GridUtils.java  |   19 +-
 .../GridTcpCommunicationMessageFactory.java     |    5 +-
 .../internal/visor/cache/VisorCacheMetrics.java |   37 +-
 .../visor/cache/VisorCacheResetMetricsTask.java |    2 +-
 .../internal/visor/node/VisorNodeGcTask.java    |    2 +-
 .../ignite/interop/InteropConfiguration.java    |   30 +
 .../java/org/apache/ignite/interop/package.html |   15 +
 .../ignite/mxbean/CacheMetricsMXBean.java       |  245 +++
 .../mxbean/ClusterLocalNodeMetricsMXBean.java   |  240 +++
 .../apache/ignite/mxbean/IgniteCacheMXBean.java |   72 -
 .../org/apache/ignite/mxbean/IgniteMBean.java   |  386 -----
 .../ignite/mxbean/IgniteMBeanDescription.java   |   34 -
 .../IgniteMBeanParametersDescriptions.java      |   34 -
 .../mxbean/IgniteMBeanParametersNames.java      |   34 -
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |  386 +++++
 .../ignite/mxbean/IgniteMXBeanDescription.java  |   34 +
 .../IgniteMXBeanParametersDescriptions.java     |   34 +
 .../mxbean/IgniteMXBeanParametersNames.java     |   34 +
 .../ignite/mxbean/IgniteStandardMBean.java      |  275 ----
 .../ignite/mxbean/IgniteThreadPoolMBean.java    |  152 --
 .../ignite/mxbean/IgniteThreadPoolMXBean.java   |  152 ++
 .../org/apache/ignite/mxbean/IgnitionMBean.java |  151 --
 .../apache/ignite/mxbean/IgnitionMXBean.java    |  151 ++
 .../ignite/spi/IgniteSpiManagementMBean.java    |   14 +-
 .../noop/NoopAuthenticationSpiMBean.java        |    2 +-
 .../cache/CacheCheckpointSpiMBean.java          |    4 +-
 .../checkpoint/jdbc/JdbcCheckpointSpiMBean.java |   24 +-
 .../sharedfs/SharedFsCheckpointSpiMBean.java    |    6 +-
 .../fifoqueue/FifoQueueCollisionSpiMBean.java   |   18 +-
 .../JobStealingCollisionSpiMBean.java           |   36 +-
 .../PriorityQueueCollisionSpiMBean.java         |   34 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   58 +-
 .../local/LocalDeploymentSpiMBean.java          |    2 +-
 .../spi/discovery/DiscoveryMetricsHelper.java   |  368 -----
 .../spi/discovery/DiscoveryMetricsProvider.java |    2 +-
 .../discovery/DiscoveryNodeMetricsAdapter.java  | 1068 -------------
 .../discovery/tcp/TcpClientDiscoverySpi.java    |    6 +-
 .../tcp/TcpClientDiscoverySpiMBean.java         |   34 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   12 +-
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |   62 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   16 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   36 +-
 .../memory/MemoryEventStorageSpiMBean.java      |   10 +-
 .../failover/always/AlwaysFailoverSpiMBean.java |    6 +-
 .../JobStealingFailoverSpiMBean.java            |    8 +-
 .../failover/never/NeverFailoverSpiMBean.java   |    2 +-
 .../adaptive/AdaptiveCpuLoadProbe.java          |    2 +-
 .../adaptive/AdaptiveJobCountLoadProbe.java     |    2 +-
 .../adaptive/AdaptiveLoadBalancingSpiMBean.java |    4 +-
 .../AdaptiveProcessingTimeLoadProbe.java        |    2 +-
 .../RoundRobinLoadBalancingSpiMBean.java        |    4 +-
 .../WeightedRandomLoadBalancingSpiMBean.java    |    6 +-
 .../noop/NoopSecureSessionSpiMBean.java         |    2 +-
 .../swapspace/file/FileSwapSpaceSpiMBean.java   |   12 +-
 .../apache/ignite/streamer/StreamerMBean.java   |   34 +-
 .../ignite/streamer/StreamerStageMBean.java     |   24 +-
 .../ignite/streamer/StreamerWindowMBean.java    |   10 +-
 .../index/StreamerIndexProviderMBean.java       |   12 +-
 .../transactions/IgniteTxConcurrency.java       |    2 +-
 modules/core/src/test/config/ggfs-loopback.xml  |   41 +-
 .../core/src/test/config/ggfs-no-endpoint.xml   |  199 ---
 modules/core/src/test/config/ggfs-shmem.xml     |   41 +-
 .../apache/ignite/IgniteCacheAffinityTest.java  |  266 ++++
 .../ignite/internal/ClusterMetricsSelfTest.java |  124 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |    6 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    2 +-
 .../internal/GridNonHistoryMetricsSelfTest.java |    4 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |    2 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |  752 +++++++++-
 ...cheTransactionalAbstractMetricsSelfTest.java |   20 +-
 .../cache/GridCacheVersionSelfTest.java         |   23 +-
 ...ePartitionedNearDisabledMetricsSelfTest.java |   14 +-
 ...AtomicPartitionedTckMetricsSelfTestImpl.java |  183 +++
 .../near/GridCacheNearMetricsSelfTest.java      |  163 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |   38 +-
 ...dCacheAtomicLocalTckMetricsSelfTestImpl.java |  161 ++
 .../ignite/p2p/GridP2PClassLoadingSelfTest.java |    2 +-
 ...bStealingCollisionSpiAttributesSelfTest.java |    4 +-
 ...alingCollisionSpiCustomTopologySelfTest.java |    4 +-
 .../GridJobStealingCollisionSpiSelfTest.java    |    4 +-
 ...ClusterMetricsSnapshotSerializeSelfTest.java |  126 ++
 .../GridDiscoveryMetricsHelperSelfTest.java     |  149 --
 .../testframework/GridSpiTestContext.java       |    6 +-
 .../ignite/testframework/GridTestNode.java      |    8 +-
 .../testframework/junits/GridTestIgnite.java    |    6 +
 .../junits/spi/GridSpiAbstractTest.java         |    2 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    4 +-
 .../testsuites/IgniteComputeGridTestSuite.java  |    2 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |    2 +-
 .../util/GridTopologyHeapSizeSelfTest.java      |    4 +-
 .../ignite/util/mbeans/GridMBeanSelfTest.java   |   49 +-
 .../query/h2/GridH2ResultSetIterator.java       |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  173 ++-
 .../query/h2/opt/GridH2IndexBase.java           |    5 +-
 .../query/h2/sql/GridSqlAggregateFunction.java  |   76 +
 .../processors/query/h2/sql/GridSqlAlias.java   |   55 +
 .../processors/query/h2/sql/GridSqlColumn.java  |   57 +
 .../processors/query/h2/sql/GridSqlConst.java   |   39 +
 .../processors/query/h2/sql/GridSqlElement.java |   73 +
 .../query/h2/sql/GridSqlFunction.java           |  142 ++
 .../query/h2/sql/GridSqlFunctionType.java       |   78 +
 .../processors/query/h2/sql/GridSqlJoin.java    |   68 +
 .../query/h2/sql/GridSqlOperation.java          |   73 +
 .../query/h2/sql/GridSqlOperationType.java      |  202 +++
 .../query/h2/sql/GridSqlParameter.java          |   44 +
 .../query/h2/sql/GridSqlQueryParser.java        |  495 +++++++
 .../query/h2/sql/GridSqlQuerySplitter.java      |  252 ++++
 .../processors/query/h2/sql/GridSqlSelect.java  |  287 ++++
 .../query/h2/sql/GridSqlSubquery.java           |   44 +
 .../processors/query/h2/sql/GridSqlTable.java   |   55 +
 .../processors/query/h2/sql/GridSqlValue.java   |   17 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  270 ++++
 .../query/h2/twostep/GridMergeIndex.java        |  290 ++++
 .../h2/twostep/GridMergeIndexUnsorted.java      |   85 ++
 .../query/h2/twostep/GridMergeTable.java        |  178 +++
 .../h2/twostep/GridReduceQueryExecutor.java     |  242 +++
 .../query/h2/twostep/GridResultPage.java        |   59 +
 .../twostep/messages/GridNextPageRequest.java   |   59 +
 .../twostep/messages/GridNextPageResponse.java  |  180 +++
 .../query/h2/twostep/messages/GridQueryAck.java |   34 +
 .../twostep/messages/GridQueryFailResponse.java |   46 +
 .../h2/twostep/messages/GridQueryRequest.java   |   61 +
 .../cache/GridCacheCrossCacheQuerySelfTest.java |  111 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |    5 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  289 ++++
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +
 .../test/resources/spring-ping-pong-partner.xml |   38 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    6 +
 .../uri/GridUriDeploymentSpiMBean.java          |    8 +-
 .../commands/alert/VisorAlertCommand.scala      |    7 +-
 .../commands/top/VisorTopologyCommand.scala     |   11 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    6 +-
 pom.xml                                         |  132 +-
 231 files changed, 13838 insertions(+), 8565 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/99e3fa61/pom.xml
----------------------------------------------------------------------


[06/12] incubator-ignite git commit: # IGNITE-102: Replace GridGain to Ignite.

Posted by ak...@apache.org.
# IGNITE-102: Replace GridGain to Ignite.


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

Branch: refs/heads/sprint-1
Commit: c2c2e7ffa3d3e773880a0c2dd8928f031998b2fc
Parents: 786193f
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jan 28 16:21:32 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jan 28 16:21:32 2015 +0700

----------------------------------------------------------------------
 modules/yardstick/README.md                     |  2 +-
 .../config/benchmark-atomic-win.properties      |  4 ++--
 .../config/benchmark-atomic.properties          |  4 ++--
 .../config/benchmark-compute-win.properties     | 10 ++++----
 .../config/benchmark-compute.properties         | 10 ++++----
 .../config/benchmark-query-win.properties       |  6 ++---
 .../yardstick/config/benchmark-query.properties |  6 ++---
 .../config/benchmark-tx-win.properties          |  4 ++--
 .../yardstick/config/benchmark-tx.properties    |  4 ++--
 .../yardstick/config/benchmark-win.properties   | 14 ++++++------
 modules/yardstick/config/benchmark.properties   | 24 ++++++++++----------
 modules/yardstick/config/ignite-base-config.xml | 16 ++++++-------
 .../yardstick/IgniteAbstractBenchmark.java      |  2 +-
 .../yardstick/IgniteBenchmarkArguments.java     |  8 +++----
 .../org/apache/ignite/yardstick/IgniteNode.java |  4 ++--
 .../cache/IgniteCacheAbstractBenchmark.java     |  2 +-
 .../yardstick/cache/IgniteGetBenchmark.java     |  2 +-
 .../yardstick/cache/IgnitePutBenchmark.java     |  2 +-
 .../yardstick/cache/IgnitePutGetBenchmark.java  |  2 +-
 .../cache/IgnitePutGetTxBenchmark.java          |  2 +-
 .../yardstick/cache/IgnitePutTxBenchmark.java   |  2 +-
 .../cache/IgniteSqlQueryBenchmark.java          |  2 +-
 .../cache/IgniteSqlQueryJoinBenchmark.java      |  2 +-
 .../cache/IgniteSqlQueryPutBenchmark.java       |  2 +-
 .../compute/IgniteAffinityCallBenchmark.java    |  2 +-
 .../yardstick/compute/IgniteApplyBenchmark.java |  2 +-
 .../compute/IgniteBroadcastBenchmark.java       |  2 +-
 .../compute/IgniteExecuteBenchmark.java         |  2 +-
 .../yardstick/compute/IgniteRunBenchmark.java   |  2 +-
 29 files changed, 73 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/README.md
----------------------------------------------------------------------
diff --git a/modules/yardstick/README.md b/modules/yardstick/README.md
index d2aebc0..660c453 100644
--- a/modules/yardstick/README.md
+++ b/modules/yardstick/README.md
@@ -37,7 +37,7 @@ The following Ignite benchmark properties can be defined in the benchmark config
 
 * `-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
+* `-cfg <path>` or `--Config <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`)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-atomic-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic-win.properties b/modules/yardstick/config/benchmark-atomic-win.properties
index 96dde32..d816e62 100644
--- a/modules/yardstick/config/benchmark-atomic-win.properties
+++ b/modules/yardstick/config/benchmark-atomic-win.properties
@@ -38,5 +38,5 @@ set DRIVER_HOSTS=localhost
 :: 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).
 set CONFIGS=^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-atomic.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic.properties b/modules/yardstick/config/benchmark-atomic.properties
index 1f93d5c..aaa9946 100644
--- a/modules/yardstick/config/benchmark-atomic.properties
+++ b/modules/yardstick/config/benchmark-atomic.properties
@@ -43,6 +43,6 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS}
 # 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="\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-compute-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute-win.properties b/modules/yardstick/config/benchmark-compute-win.properties
index 35e8873..5c41110 100644
--- a/modules/yardstick/config/benchmark-compute-win.properties
+++ b/modules/yardstick/config/benchmark-compute-win.properties
@@ -38,8 +38,8 @@ set DRIVER_HOSTS=localhost
 :: 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).
 set CONFIGS=^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-compute.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute.properties b/modules/yardstick/config/benchmark-compute.properties
index 1f38eb6..45aadcb 100644
--- a/modules/yardstick/config/benchmark-compute.properties
+++ b/modules/yardstick/config/benchmark-compute.properties
@@ -43,9 +43,9 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS}
 # 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="\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-query-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query-win.properties b/modules/yardstick/config/benchmark-query-win.properties
index a2c102c..6164038 100644
--- a/modules/yardstick/config/benchmark-query-win.properties
+++ b/modules/yardstick/config/benchmark-query-win.properties
@@ -38,6 +38,6 @@ set DRIVER_HOSTS=localhost
 :: 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).
 set CONFIGS=^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-query.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query.properties b/modules/yardstick/config/benchmark-query.properties
index c3e7a6e..18440af 100644
--- a/modules/yardstick/config/benchmark-query.properties
+++ b/modules/yardstick/config/benchmark-query.properties
@@ -43,7 +43,7 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS}
 # 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="\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-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 IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-tx-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx-win.properties b/modules/yardstick/config/benchmark-tx-win.properties
index 7b12202..9d01d47 100644
--- a/modules/yardstick/config/benchmark-tx-win.properties
+++ b/modules/yardstick/config/benchmark-tx-win.properties
@@ -38,5 +38,5 @@ set DRIVER_HOSTS=localhost
 :: 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).
 set CONFIGS=^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-tx.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx.properties b/modules/yardstick/config/benchmark-tx.properties
index c3d69f2..2f22f65 100644
--- a/modules/yardstick/config/benchmark-tx.properties
+++ b/modules/yardstick/config/benchmark-tx.properties
@@ -43,6 +43,6 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS}
 # 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="\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-win.properties b/modules/yardstick/config/benchmark-win.properties
index 00a3c17..740a132 100644
--- a/modules/yardstick/config/benchmark-win.properties
+++ b/modules/yardstick/config/benchmark-win.properties
@@ -41,10 +41,10 @@ set DRIVER_HOSTS=localhost
 :: 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).
 set CONFIGS=^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^
--ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^
+-cfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/config/benchmark.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark.properties b/modules/yardstick/config/benchmark.properties
index 034eb46..b731953 100644
--- a/modules/yardstick/config/benchmark.properties
+++ b/modules/yardstick/config/benchmark.properties
@@ -46,16 +46,16 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS}
 # 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="\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,\
--ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-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 IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup\
 "

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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 16d0a70..c92f630 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -28,17 +28,17 @@
         <property name="peerClassLoadingEnabled" value="false"/>
 
         <!--property name="marshaller">
-            <bean class="org.gridgain.grid.marshaller.optimized.GridOptimizedMarshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.IgniteOptimizedMarshaller">
                 <property name="requireSerializable" value="true"/>
                 <property name="classNames">
                     <list>
-                        <value>org.yardstickframework.gridgain.cache.model.SampleValue</value>
-                        <value>org.yardstickframework.gridgain.cache.model.Person</value>
-                        <value>org.yardstickframework.gridgain.cache.model.Organization</value>
-                        <value>org.yardstickframework.gridgain.compute.model.NoopTask$NoopJob</value>
-                        <value>org.yardstickframework.gridgain.compute.model.NoopCallable</value>
-                        <value>org.yardstickframework.gridgain.compute.GridGainRunBenchmark$NoopRunnable</value>
-                        <value>org.yardstickframework.gridgain.compute.GridGainApplyBenchmark$NoopClosure</value>
+                        <value>org.apache.ignite.yardstick.cache.model.SampleValue</value>
+                        <value>org.apache.ignite.yardstick.cache.model.Person</value>
+                        <value>org.apache.ignite.yardstick.cache.model.Organization</value>
+                        <value>org.apache.ignite.yardstick.compute.model.NoopTask$NoopJob</value>
+                        <value>org.apache.ignite.yardstick.compute.model.NoopCallable</value>
+                        <value>org.apache.ignite.yardstick.compute.IgniteRunBenchmark$NoopRunnable</value>
+                        <value>org.apache.ignite.yardstick.compute.IgniteApplyBenchmark$NoopClosure</value>
                     </list>
                 </property>
             </bean>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index cca6935..6f9f903 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
@@ -29,7 +29,7 @@ import static org.apache.ignite.events.IgniteEventType.*;
 import static org.yardstickframework.BenchmarkUtils.*;
 
 /**
- * Abstract class for GridGain benchmarks.
+ * Abstract class for Ignite benchmarks.
  */
 public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
     /** Arguments. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index ebcddf3..1204720 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -23,7 +23,7 @@ import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.transactions.*;
 
 /**
- * Input arguments for GridGain benchmarks.
+ * Input arguments for Ignite benchmarks.
  */
 @SuppressWarnings({"UnusedDeclaration", "FieldCanBeLocal"})
 public class IgniteBenchmarkArguments {
@@ -35,8 +35,8 @@ public class IgniteBenchmarkArguments {
     @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 = {"-cfg", "--Config"}, description = "Configuration file")
+    private String cfg = "config/ignite-localhost-config.xml";
 
     /** */
     @Parameter(names = {"-sm", "--syncMode"}, description = "Synchronization mode")
@@ -188,7 +188,7 @@ public class IgniteBenchmarkArguments {
      * @return Configuration file.
      */
     public String configuration() {
-        return ggcfg;
+        return cfg;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index b80e7bf..ad71ca2 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -36,7 +36,7 @@ import static org.apache.ignite.cache.CacheDistributionMode.*;
 import static org.apache.ignite.cache.CacheMemoryMode.*;
 
 /**
- * Standalone GridGain node.
+ * Standalone Ignite node.
  */
 public class IgniteNode implements BenchmarkServer {
     /** Grid instance. */
@@ -73,7 +73,7 @@ public class IgniteNode implements BenchmarkServer {
 
         for (CacheConfiguration cc : c.getCacheConfiguration()) {
             // IgniteNode can not run in CLIENT_ONLY mode,
-            // except the case when it's used inside GridGainAbstractBenchmark.
+            // except the case when it's used inside IgniteAbstractBenchmark.
             CacheDistributionMode distroMode = args.distributionMode() == CLIENT_ONLY && !clientMode ?
                 PARTITIONED_ONLY : args.distributionMode();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index 636057c..7b3d202 100644
--- 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
@@ -22,7 +22,7 @@ import org.apache.ignite.yardstick.*;
 import org.yardstickframework.*;
 
 /**
- * Abstract class for GridGain benchmarks which use cache.
+ * Abstract class for Ignite benchmarks which use cache.
  */
 public abstract class IgniteCacheAbstractBenchmark extends IgniteAbstractBenchmark {
     /** Cache. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
index 30a8eb5..783523b 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
@@ -22,7 +22,7 @@ import org.apache.ignite.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs get operations.
+ * Ignite benchmark that performs get operations.
  */
 public class IgniteGetBenchmark extends IgniteCacheAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
index 5136bde..ee55d57 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
@@ -23,7 +23,7 @@ import org.apache.ignite.yardstick.cache.model.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs put operations.
+ * Ignite benchmark that performs put operations.
  */
 public class IgnitePutBenchmark extends IgniteCacheAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
index 2b9fa02..73b183f 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
@@ -23,7 +23,7 @@ import org.apache.ignite.yardstick.cache.model.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs put and get operations.
+ * Ignite benchmark that performs put and get operations.
  */
 public class IgnitePutGetBenchmark extends IgniteCacheAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
index 30acb41..a274b5b 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
@@ -24,7 +24,7 @@ import org.apache.ignite.yardstick.cache.model.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs transactional put and get operations.
+ * Ignite benchmark that performs transactional put and get operations.
  */
 public class IgnitePutGetTxBenchmark extends IgniteCacheAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
index e1cefbc..1f92392 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
@@ -23,7 +23,7 @@ import org.apache.ignite.yardstick.cache.model.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs transactional put operations.
+ * Ignite benchmark that performs transactional put operations.
  */
 public class IgnitePutTxBenchmark extends IgniteCacheAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
index 38aa632..3bca3d6 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
@@ -28,7 +28,7 @@ import java.util.concurrent.*;
 import static org.yardstickframework.BenchmarkUtils.*;
 
 /**
- * GridGain benchmark that performs query operations.
+ * Ignite benchmark that performs query operations.
  */
 public class IgniteSqlQueryBenchmark extends IgniteCacheAbstractBenchmark {
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
index 7b4262b..5f8cc77 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
@@ -28,7 +28,7 @@ import java.util.concurrent.*;
 import static org.yardstickframework.BenchmarkUtils.*;
 
 /**
- * GridGain benchmark that performs query operations with joins.
+ * Ignite benchmark that performs query operations with joins.
  */
 public class IgniteSqlQueryJoinBenchmark extends IgniteCacheAbstractBenchmark {
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
index 9d8ef4f..fcb8f2e 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
@@ -26,7 +26,7 @@ import java.util.*;
 import java.util.concurrent.*;
 
 /**
- * GridGain benchmark that performs put and query operations.
+ * Ignite benchmark that performs put and query operations.
  */
 public class IgniteSqlQueryPutBenchmark extends IgniteCacheAbstractBenchmark {
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index 28ac816..7008d52 100644
--- 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
@@ -24,7 +24,7 @@ import java.util.*;
 import java.util.concurrent.*;
 
 /**
- * GridGain benchmark that performs affinity call operations.
+ * Ignite benchmark that performs affinity call operations.
  */
 public class IgniteAffinityCallBenchmark extends IgniteAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index 0b40cdd..816d0d0 100644
--- 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
@@ -25,7 +25,7 @@ import java.io.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs apply operations.
+ * Ignite benchmark that performs apply operations.
  */
 public class IgniteApplyBenchmark extends IgniteAbstractBenchmark {
     /** Args for apply. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index 0e37efa..1be6c44 100644
--- 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
@@ -23,7 +23,7 @@ import org.apache.ignite.yardstick.compute.model.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs broadcast operations.
+ * Ignite benchmark that performs broadcast operations.
  */
 public class IgniteBroadcastBenchmark extends IgniteAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index 4fc4530..125ea6a 100644
--- 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
@@ -23,7 +23,7 @@ import org.apache.ignite.yardstick.compute.model.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs execute operations.
+ * Ignite benchmark that performs execute operations.
  */
 public class IgniteExecuteBenchmark extends IgniteAbstractBenchmark {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c2c2e7ff/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
index d0ca63e..b9b43b3 100644
--- 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
@@ -24,7 +24,7 @@ import java.io.*;
 import java.util.*;
 
 /**
- * GridGain benchmark that performs run operations.
+ * Ignite benchmark that performs run operations.
  */
 public class IgniteRunBenchmark extends IgniteAbstractBenchmark {
     /** Jobs for run */


[08/12] incubator-ignite git commit: # IGNITE-102: Renaming + debug.

Posted by ak...@apache.org.
# IGNITE-102: Renaming + debug.


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

Branch: refs/heads/sprint-1
Commit: fa4ac59ac62e5fe2a14405e1cac51bfe94f5cb58
Parents: 641a4c8
Author: AKuznetsov <ak...@gridgain.com>
Authored: Thu Jan 29 10:13:29 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Thu Jan 29 10:13:29 2015 +0700

----------------------------------------------------------------------
 .../ignite/yardstick/IgniteAbstractBenchmark.java       |  6 +++---
 .../ignite/yardstick/cache/IgniteGetBenchmark.java      |  2 +-
 .../ignite/yardstick/cache/IgnitePutBenchmark.java      |  2 +-
 .../ignite/yardstick/cache/IgnitePutGetBenchmark.java   |  2 +-
 .../ignite/yardstick/cache/IgnitePutGetTxBenchmark.java | 12 ++++++++++--
 .../ignite/yardstick/cache/IgnitePutTxBenchmark.java    |  2 +-
 .../ignite/yardstick/cache/IgniteSqlQueryBenchmark.java |  4 ++--
 .../yardstick/cache/IgniteSqlQueryJoinBenchmark.java    |  4 ++--
 .../yardstick/cache/IgniteSqlQueryPutBenchmark.java     |  2 +-
 .../yardstick/compute/IgniteAffinityCallBenchmark.java  |  2 +-
 .../ignite/yardstick/compute/IgniteApplyBenchmark.java  |  2 +-
 .../yardstick/compute/IgniteBroadcastBenchmark.java     |  2 +-
 .../yardstick/compute/IgniteExecuteBenchmark.java       |  2 +-
 .../ignite/yardstick/compute/IgniteRunBenchmark.java    |  2 +-
 14 files changed, 27 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/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
index 6f9f903..c09d185 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
@@ -78,7 +78,7 @@ public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
     /**
      * @return Grid.
      */
-    protected Ignite grid() {
+    protected Ignite ignite() {
         return node.ignite();
     }
 
@@ -88,7 +88,7 @@ public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
     private void waitForNodes() throws Exception {
         final CountDownLatch nodesStartedLatch = new CountDownLatch(1);
 
-        grid().events().localListen(new IgnitePredicate<IgniteEvent>() {
+        ignite().events().localListen(new IgnitePredicate<IgniteEvent>() {
             @Override public boolean apply(IgniteEvent gridEvt) {
                 if (nodesStarted())
                     nodesStartedLatch.countDown();
@@ -108,7 +108,7 @@ public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
      * @return {@code True} if all nodes are started, {@code false} otherwise.
      */
     private boolean nodesStarted() {
-        return grid().cluster().nodes().size() >= args.nodes();
+        return ignite().cluster().nodes().size() >= args.nodes();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
index 783523b..4e4ee9d 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
@@ -36,6 +36,6 @@ public class IgniteGetBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("atomic");
+        return ignite().jcache("atomic");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
index ee55d57..1993ac8 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
@@ -37,6 +37,6 @@ public class IgnitePutBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("atomic");
+        return ignite().jcache("atomic");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
index 73b183f..cb6e220 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
@@ -42,6 +42,6 @@ public class IgnitePutGetBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("atomic");
+        return ignite().jcache("atomic");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
index a274b5b..944d6a4 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
@@ -31,15 +31,23 @@ public class IgnitePutGetTxBenchmark extends IgniteCacheAbstractBenchmark {
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
         int key = nextRandom(0, args.range() / 2);
 
-        try (IgniteTx tx = grid().transactions().txStart()) {
+        ignite().log().debug("IgnitePutGetTxBenchmark - key: " + key);
+
+        try (IgniteTx tx = ignite().transactions().txStart()) {
+            ignite().log().debug("IgnitePutGetTxBenchmark: txStart()");
+
             Object val = cache.get(key);
 
+            ignite().log().debug("IgnitePutGetTxBenchmark - get():" + val);
+
             if (val != null)
                 key = nextRandom(args.range() / 2, args.range());
 
             cache.put(key, new SampleValue(key));
 
             tx.commit();
+
+            ignite().log().debug("IgnitePutGetTxBenchmark - txCommit");
         }
 
         return true;
@@ -47,6 +55,6 @@ public class IgnitePutGetTxBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("tx");
+        return ignite().jcache("tx");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
index 1f92392..ebf4f20 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
@@ -38,6 +38,6 @@ public class IgnitePutTxBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("tx");
+        return ignite().jcache("tx");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
index 3bca3d6..967197f 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
@@ -42,7 +42,7 @@ public class IgniteSqlQueryBenchmark extends IgniteCacheAbstractBenchmark {
 
         long start = System.nanoTime();
 
-        try (IgniteDataLoader<Integer, Person> dataLdr = grid().dataLoader(cache.getName())) {
+        try (IgniteDataLoader<Integer, Person> dataLdr = ignite().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));
 
@@ -89,6 +89,6 @@ public class IgniteSqlQueryBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("query");
+        return ignite().jcache("query");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
index 5f8cc77..d74c56d 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
@@ -42,7 +42,7 @@ public class IgniteSqlQueryJoinBenchmark extends IgniteCacheAbstractBenchmark {
 
         long start = System.nanoTime();
 
-        try (IgniteDataLoader<Object, Object> dataLdr = grid().dataLoader(cache.getName())) {
+        try (IgniteDataLoader<Object, Object> dataLdr = ignite().dataLoader(cache.getName())) {
             final int orgRange = args.range() / 10;
 
             // Populate organizations.
@@ -113,6 +113,6 @@ public class IgniteSqlQueryJoinBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("query");
+        return ignite().jcache("query");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
index fcb8f2e..d4a7221 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
@@ -81,6 +81,6 @@ public class IgniteSqlQueryPutBenchmark extends IgniteCacheAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override protected IgniteCache<Integer, Object> cache() {
-        return grid().jcache("query");
+        return ignite().jcache("query");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/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
index 7008d52..eeff847 100644
--- 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
@@ -29,7 +29,7 @@ import java.util.concurrent.*;
 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());
+        ignite().compute().affinityCall("compute", ThreadLocalRandom.current().nextInt(), new NoopCallable());
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/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
index 816d0d0..d42b356 100644
--- 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
@@ -45,7 +45,7 @@ public class IgniteApplyBenchmark extends IgniteAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        grid().compute().apply(new NoopClosure(), applyArgs);
+        ignite().compute().apply(new NoopClosure(), applyArgs);
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/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
index 1be6c44..0104a7b 100644
--- 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
@@ -28,7 +28,7 @@ import java.util.*;
 public class IgniteBroadcastBenchmark extends IgniteAbstractBenchmark {
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        grid().compute().broadcast(new NoopCallable());
+        ignite().compute().broadcast(new NoopCallable());
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/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
index 125ea6a..c2dc4d4 100644
--- 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
@@ -28,7 +28,7 @@ import java.util.*;
 public class IgniteExecuteBenchmark extends IgniteAbstractBenchmark {
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        grid().compute().execute(new NoopTask(args.jobs()), null);
+        ignite().compute().execute(new NoopTask(args.jobs()), null);
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa4ac59a/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
index b9b43b3..0da0670 100644
--- 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
@@ -44,7 +44,7 @@ public class IgniteRunBenchmark extends IgniteAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        grid().compute().run(jobs);
+        ignite().compute().run(jobs);
         
         return true;
     }


[03/12] incubator-ignite git commit: # IGNITE-102: Added configs.

Posted by ak...@apache.org.
# IGNITE-102: Added configs.


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

Branch: refs/heads/sprint-1
Commit: c9aadf0778d818b00fe20d549dc81f7082d79652
Parents: 99e3fa6
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jan 28 13:05:49 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jan 28 13:05:49 2015 +0700

----------------------------------------------------------------------
 .../config/benchmark-atomic-win.properties      |  42 +++++++
 .../config/benchmark-atomic.properties          |  48 ++++++++
 .../config/benchmark-compute-win.properties     |  45 +++++++
 .../config/benchmark-compute.properties         |  51 ++++++++
 .../config/benchmark-query-win.properties       |  43 +++++++
 .../yardstick/config/benchmark-query.properties |  49 ++++++++
 .../config/benchmark-tx-win.properties          |  42 +++++++
 .../yardstick/config/benchmark-tx.properties    |  48 ++++++++
 .../yardstick/config/benchmark-win.properties   |  50 ++++++++
 modules/yardstick/config/benchmark.properties   |  61 ++++++++++
 modules/yardstick/config/ignite-base-config.xml | 118 +++++++++++++++++++
 .../config/ignite-localhost-config.xml          |  55 +++++++++
 .../config/ignite-multicast-config.xml          |  38 ++++++
 .../yardstick/cache/GetBenchmarkIgnite.java     |  41 -------
 .../yardstick/cache/IgniteGetBenchmark.java     |  41 +++++++
 15 files changed, 731 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-atomic-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic-win.properties b/modules/yardstick/config/benchmark-atomic-win.properties
new file mode 100644
index 0000000..97132f0
--- /dev/null
+++ b/modules/yardstick/config/benchmark-atomic-win.properties
@@ -0,0 +1,42 @@
+::
+:: Contains benchmarks for ATOMIC cache.
+::
+
+:: JVM options.
+set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
+
+:: Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+:: set JVM_OPTS=%JVM_OPTS%^
+::  -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.
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+:: Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework
+
+:: 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.
+set SERVER_HOSTS=localhost,localhost
+
+:: Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+set DRIVER_HOSTS=localhost
+
+:: Remote username.
+:: set REMOTE_USER=
+
+:: 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).
+set CONFIGS=^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-atomic.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic.properties b/modules/yardstick/config/benchmark-atomic.properties
new file mode 100644
index 0000000..d10259d
--- /dev/null
+++ b/modules/yardstick/config/benchmark-atomic.properties
@@ -0,0 +1,48 @@
+#
+# Contains benchmarks for ATOMIC cache.
+#
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+# JVM_OPTS=${JVM_OPTS}" \
+#  -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
+
+# 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="\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-compute-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute-win.properties b/modules/yardstick/config/benchmark-compute-win.properties
new file mode 100644
index 0000000..06c7f6a
--- /dev/null
+++ b/modules/yardstick/config/benchmark-compute-win.properties
@@ -0,0 +1,45 @@
+::
+:: Contains benchmarks for distributed computations.
+::
+
+:: JVM options.
+set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
+
+:: Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+:: set JVM_OPTS=%JVM_OPTS%^
+::  -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.
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+:: Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework
+
+:: 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.
+set SERVER_HOSTS=localhost,localhost
+
+:: Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+set DRIVER_HOSTS=localhost
+
+:: Remote username.
+:: set REMOTE_USER=
+
+:: 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).
+set CONFIGS=^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 1 -d 30 -t 1 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-compute.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute.properties b/modules/yardstick/config/benchmark-compute.properties
new file mode 100644
index 0000000..d90a954
--- /dev/null
+++ b/modules/yardstick/config/benchmark-compute.properties
@@ -0,0 +1,51 @@
+#
+# Contains benchmarks for distributed computations.
+#
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+# JVM_OPTS=${JVM_OPTS}" \
+#  -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
+
+# 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="\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-query-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query-win.properties b/modules/yardstick/config/benchmark-query-win.properties
new file mode 100644
index 0000000..377bac0
--- /dev/null
+++ b/modules/yardstick/config/benchmark-query-win.properties
@@ -0,0 +1,43 @@
+::
+:: Contains benchmarks for SQL queries.
+::
+
+:: JVM options.
+:: set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
+
+:: Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+set JVM_OPTS=%JVM_OPTS%^
+  -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.
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+:: Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework
+
+:: 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.
+set SERVER_HOSTS=localhost,localhost
+
+:: Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+set DRIVER_HOSTS=localhost
+
+:: Remote username.
+:: set REMOTE_USER=
+
+:: 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).
+set CONFIGS=^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-query.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query.properties b/modules/yardstick/config/benchmark-query.properties
new file mode 100644
index 0000000..b467e0b
--- /dev/null
+++ b/modules/yardstick/config/benchmark-query.properties
@@ -0,0 +1,49 @@
+#
+# Contains benchmarks for SQL queries.
+#
+
+# JVM options.
+# JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_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
+
+# 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="\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-tx-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx-win.properties b/modules/yardstick/config/benchmark-tx-win.properties
new file mode 100644
index 0000000..7e098a5
--- /dev/null
+++ b/modules/yardstick/config/benchmark-tx-win.properties
@@ -0,0 +1,42 @@
+::
+:: Contains benchmarks for TRANSACTIONAL cache.
+::
+
+:: JVM options.
+set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
+
+:: Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+:: set JVM_OPTS=%JVM_OPTS%^
+::  -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.
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+:: Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework
+
+:: 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.
+set SERVER_HOSTS=localhost,localhost
+
+:: Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+set DRIVER_HOSTS=localhost
+
+:: Remote username.
+:: set REMOTE_USER=
+
+:: 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).
+set CONFIGS=^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-tx.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx.properties b/modules/yardstick/config/benchmark-tx.properties
new file mode 100644
index 0000000..83be491
--- /dev/null
+++ b/modules/yardstick/config/benchmark-tx.properties
@@ -0,0 +1,48 @@
+#
+# Contains benchmarks for TRANSACTIONAL cache.
+#
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+# JVM_OPTS=${JVM_OPTS}" \
+#  -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, comma separated.
+# 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, comma separated.
+BENCHMARK_PACKAGES=org.yardstickframework
+
+# 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="\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-win.properties b/modules/yardstick/config/benchmark-win.properties
new file mode 100644
index 0000000..ba77c84
--- /dev/null
+++ b/modules/yardstick/config/benchmark-win.properties
@@ -0,0 +1,50 @@
+::
+:: Contains all benchmarks for:
+:: - ATOMIC cache
+:: - TRANSACTIONAL cache
+:: - SQL queries
+::
+
+:: JVM options.
+set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
+
+:: Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+:: set JVM_OPTS=%JVM_OPTS%^
+::  -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.
+BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+:: Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework
+
+:: 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.
+set SERVER_HOSTS=localhost,localhost
+
+:: Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+set DRIVER_HOSTS=localhost
+
+:: Remote username.
+:: set REMOTE_USER=
+
+:: 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).
+set CONFIGS=^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,^
+-ggcfg %SCRIPT_DIR%\..\config\ignite-localhost-config.xml -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/benchmark.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark.properties b/modules/yardstick/config/benchmark.properties
new file mode 100644
index 0000000..bbe9ff0
--- /dev/null
+++ b/modules/yardstick/config/benchmark.properties
@@ -0,0 +1,61 @@
+#
+# Contains all benchmarks for:
+# - ATOMIC cache
+# - TRANSACTIONAL cache
+# - SQL queries
+#
+
+# JVM options.
+JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+# JVM_OPTS=${JVM_OPTS}" \
+#  -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
+
+# 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 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="\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutBenchmark -sn IgniteNode -ds atomic-put-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetBenchmark -sn IgniteNode -ds atomic-put-get-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutTxBenchmark -sn IgniteNode -ds tx-put-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutGetTxBenchmark -sn IgniteNode -ds tx-put-get-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryBenchmark -sn IgniteNode -ds sql-query-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryJoinBenchmark -sn IgniteNode -ds sql-query-join-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgniteSqlQueryPutBenchmark -sn IgniteNode -ds sql-query-put-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteAffinityCallBenchmark -sn IgniteNode -ds affcall-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteApplyBenchmark -sn IgniteNode -ds apply-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteBroadcastBenchmark -sn IgniteNode -ds broad-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteExecuteBenchmark -sn IgniteNode -ds exec-compute-1-backup,\
+-ggcfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -j 10 -dn IgniteRunBenchmark -sn IgniteNode -ds run-compute-1-backup\
+"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/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
new file mode 100644
index 0000000..69fe9b1
--- /dev/null
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -0,0 +1,118 @@
+<?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">
+    <bean id="base-grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" abstract="true">
+        <property name="peerClassLoadingEnabled" value="false"/>
+
+        <!--property name="marshaller">
+            <bean class="org.gridgain.grid.marshaller.optimized.GridOptimizedMarshaller">
+                <property name="requireSerializable" value="true"/>
+                <property name="classNames">
+                    <list>
+                        <value>org.yardstickframework.gridgain.cache.model.SampleValue</value>
+                        <value>org.yardstickframework.gridgain.cache.model.Person</value>
+                        <value>org.yardstickframework.gridgain.cache.model.Organization</value>
+                        <value>org.yardstickframework.gridgain.compute.model.NoopTask$NoopJob</value>
+                        <value>org.yardstickframework.gridgain.compute.model.NoopCallable</value>
+                        <value>org.yardstickframework.gridgain.compute.GridGainRunBenchmark$NoopRunnable</value>
+                        <value>org.yardstickframework.gridgain.compute.GridGainApplyBenchmark$NoopClosure</value>
+                    </list>
+                </property>
+            </bean>
+        </property-->
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                    <property name="name" value="atomic"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="ATOMIC"/>
+
+                    <property name="swapEnabled" value="false"/>
+
+                    <property name="queryIndexEnabled" value="false"/>
+                </bean>
+
+                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                    <property name="name" value="tx"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+
+                    <property name="swapEnabled" value="false"/>
+
+                    <property name="queryIndexEnabled" value="false"/>
+                </bean>
+
+                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                    <property name="name" value="query"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="ATOMIC"/>
+
+                    <property name="swapEnabled" value="false"/>
+
+                    <!-- Enable query indexing for query cache. -->
+                    <property name="queryIndexEnabled" value="true"/>
+                </bean>
+
+                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                    <property name="name" value="compute"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+
+                    <property name="swapEnabled" value="false"/>
+
+                    <property name="queryIndexEnabled" value="false"/>
+                </bean>
+            </list>
+        </property>
+
+        <property name="restEnabled" value="false"/>
+
+        <property name="includeEventTypes">
+            <list/>
+        </property>
+
+        <property name="loadBalancingSpi">
+            <bean class="org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi">
+                <property name="perTask" value="false"/>
+            </bean>
+        </property>
+
+        <property name="communicationSpi">
+            <bean class="org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi">
+                <property name="sharedMemoryPort" value="-1"/>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/ignite-localhost-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-localhost-config.xml b/modules/yardstick/config/ignite-localhost-config.xml
new file mode 100644
index 0000000..7a51bb0
--- /dev/null
+++ b/modules/yardstick/config/ignite-localhost-config.xml
@@ -0,0 +1,55 @@
+<?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="gridgain-base-config.xml"/>
+
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-grid.cfg">
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500</value>
+                                <value>127.0.0.1:47501</value>
+                                <value>127.0.0.1:47502</value>
+                                <value>127.0.0.1:47503</value>
+                                <value>127.0.0.1:47504</value>
+                                <value>127.0.0.1:47505</value>
+                                <value>127.0.0.1:47506</value>
+                                <value>127.0.0.1:47507</value>
+                                <value>127.0.0.1:47508</value>
+                                <value>127.0.0.1:47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/modules/yardstick/config/ignite-multicast-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-multicast-config.xml b/modules/yardstick/config/ignite-multicast-config.xml
new file mode 100644
index 0000000..a81b0d5
--- /dev/null
+++ b/modules/yardstick/config/ignite-multicast-config.xml
@@ -0,0 +1,38 @@
+<?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="gridgain-base-config.xml"/>
+
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-grid.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>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c9aadf07/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
deleted file mode 100644
index bbbf4d6..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/GetBenchmarkIgnite.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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/c9aadf07/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.java
new file mode 100644
index 0000000..30a8eb5
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteGetBenchmark.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 IgniteGetBenchmark 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");
+    }
+}


[07/12] incubator-ignite git commit: Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102

Posted by ak...@apache.org.
Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-102


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

Branch: refs/heads/sprint-1
Commit: 641a4c87c91c2a260a7960e8ec942cd0bd97f2d0
Parents: c2c2e7f 673d11b
Author: AKuznetsov <ak...@gridgain.com>
Authored: Thu Jan 29 10:02:23 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Thu Jan 29 10:02:23 2015 +0700

----------------------------------------------------------------------
 .../hibernate/example-hibernate-L2-cache.xml    |  14 +-
 .../ComputeFibonacciContinuationExample.java    |   2 +-
 .../compute/ComputeRunnableExample.java         |   2 +-
 .../datagrid/CacheDataLoaderExample.java        |  15 +-
 .../HibernateL2CacheExampleNodeStartup.java     |   6 +-
 .../examples/ScalarContinuationExample.scala    |   2 +-
 .../apache/ignite/gridify/AbstractAopTest.java  |   2 +-
 .../apache/ignite/gridify/TestGridifyTask.java  |   2 +-
 .../singlesplit/SingleSplitsLoadTest.java       |   2 +-
 .../ignite/client/ClientStartNodeTask.java      |   2 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   1 +
 .../java/org/apache/ignite/IgniteCache.java     |  40 ++-
 .../java/org/apache/ignite/IgniteCluster.java   |   2 +-
 .../java/org/apache/ignite/IgniteCompute.java   |   2 +-
 .../java/org/apache/ignite/IgniteEvents.java    |   2 +-
 .../main/java/org/apache/ignite/IgniteFs.java   |   2 +-
 .../java/org/apache/ignite/IgniteManaged.java   |   2 +-
 .../java/org/apache/ignite/IgniteMessaging.java |   2 +-
 .../java/org/apache/ignite/cache/CacheLock.java |  57 ----
 .../ignite/cache/GridCacheVersionedEntry.java   |  82 ++++++
 .../org/apache/ignite/internal/GridKernal.java  |  32 +--
 .../ignite/internal/IgniteClusterAsyncImpl.java |   9 +-
 .../ignite/internal/IgniteComputeImpl.java      |  70 ++---
 .../ignite/internal/IgniteEventsImpl.java       |   9 +-
 .../ignite/internal/IgniteManagedImpl.java      |   2 +-
 .../ignite/internal/IgniteMessagingImpl.java    |  12 +-
 .../internal/executor/GridExecutorService.java  |   2 +-
 .../managers/communication/GridIoManager.java   |   3 +-
 .../processors/cache/CacheLockImpl.java         | 132 ++++++---
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../cache/GridCacheAtomicVersionComparator.java |   2 +
 .../cache/GridCacheBatchSwapEntry.java          |   1 +
 .../cache/GridCacheClearAllRunnable.java        |   1 +
 .../processors/cache/GridCacheContext.java      |  44 +++
 .../processors/cache/GridCacheEntryEx.java      |   9 +
 .../processors/cache/GridCacheEntryInfo.java    |   1 +
 .../cache/GridCacheEvictionManager.java         |   1 +
 .../cache/GridCacheEvictionRequest.java         |   1 +
 .../cache/GridCacheExplicitLockSpan.java        |   1 +
 .../processors/cache/GridCacheFuture.java       |   1 +
 .../cache/GridCacheLockTimeoutException.java    |   2 +
 .../processors/cache/GridCacheMapEntry.java     | 117 ++++++--
 .../processors/cache/GridCacheMvcc.java         |   1 +
 .../cache/GridCacheMvccCandidate.java           |   1 +
 .../processors/cache/GridCacheMvccManager.java  |   1 +
 .../cache/GridCacheOffheapSwapEntry.java        |   1 +
 .../processors/cache/GridCacheProcessor.java    |   1 +
 .../processors/cache/GridCacheProjectionEx.java |   1 +
 .../cache/GridCacheProjectionImpl.java          |   1 +
 .../processors/cache/GridCacheProxyImpl.java    |   1 +
 .../cache/GridCacheSharedContext.java           |   1 +
 .../processors/cache/GridCacheStoreManager.java |  19 +-
 .../processors/cache/GridCacheSwapEntry.java    |   1 +
 .../cache/GridCacheSwapEntryImpl.java           |   1 +
 .../processors/cache/GridCacheSwapManager.java  |   1 +
 .../processors/cache/GridCacheTtlManager.java   |   1 +
 .../cache/GridCacheUpdateAtomicResult.java      |   7 +-
 .../processors/cache/GridCacheUtils.java        |   1 +
 .../processors/cache/GridCacheVersion.java      | 256 -----------------
 .../processors/cache/GridCacheVersionEx.java    | 104 -------
 .../cache/GridCacheVersionManager.java          | 281 ------------------
 .../processors/cache/GridCacheVersionable.java  |  28 --
 .../processors/cache/GridDrResolveResult.java   |  63 -----
 .../cache/IgniteCacheExpiryPolicy.java          |   1 +
 .../processors/cache/IgniteCacheProxy.java      |  33 +--
 .../distributed/GridCacheCommittedTxInfo.java   |   2 +-
 .../distributed/GridCacheMappedVersion.java     |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |   1 +
 ...idCacheOptimisticCheckPreparedTxRequest.java |   2 +-
 ...dCacheOptimisticCheckPreparedTxResponse.java |   2 +-
 .../GridCachePerThreadTxCommitBuffer.java       |   1 +
 ...dCachePessimisticCheckCommittedTxFuture.java |   1 +
 ...CachePessimisticCheckCommittedTxRequest.java |   2 +-
 ...achePessimisticCheckCommittedTxResponse.java |   1 +
 .../distributed/GridCacheTtlUpdateRequest.java  |   1 +
 .../distributed/GridCacheTxCommitBuffer.java    |   2 +-
 .../distributed/GridDistributedBaseMessage.java |   1 +
 .../GridDistributedCacheAdapter.java            |   1 +
 .../distributed/GridDistributedCacheEntry.java  |   1 +
 .../distributed/GridDistributedLockRequest.java |   1 +
 .../GridDistributedLockResponse.java            |   1 +
 .../GridDistributedTxFinishRequest.java         |   1 +
 .../GridDistributedTxFinishResponse.java        |   1 +
 .../distributed/GridDistributedTxMapping.java   |   1 +
 .../GridDistributedTxPrepareRequest.java        |   1 +
 .../GridDistributedTxPrepareResponse.java       |   1 +
 .../GridDistributedTxRemoteAdapter.java         |  48 ++--
 .../distributed/dht/GridDhtCacheAdapter.java    |   1 +
 .../distributed/dht/GridDhtCacheEntry.java      |   1 +
 .../cache/distributed/dht/GridDhtGetFuture.java |   1 +
 .../distributed/dht/GridDhtLocalPartition.java  |   1 +
 .../distributed/dht/GridDhtLockFuture.java      |   1 +
 .../distributed/dht/GridDhtLockRequest.java     |   1 +
 .../distributed/dht/GridDhtLockResponse.java    |   1 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   1 +
 .../distributed/dht/GridDhtTxFinishFuture.java  |   1 +
 .../distributed/dht/GridDhtTxFinishRequest.java |   1 +
 .../dht/GridDhtTxFinishResponse.java            |   2 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   1 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   1 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 +
 .../dht/GridDhtTxPrepareRequest.java            |   1 +
 .../dht/GridDhtTxPrepareResponse.java           |   1 +
 .../cache/distributed/dht/GridDhtTxRemote.java  |   1 +
 .../dht/GridPartitionedGetFuture.java           |   1 +
 .../dht/atomic/GridDhtAtomicCache.java          |  11 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |   1 +
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   1 +
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   1 +
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   1 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   1 +
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   1 +
 .../atomic/GridNearAtomicUpdateResponse.java    |   1 +
 .../dht/colocated/GridDhtColocatedCache.java    |   1 +
 .../colocated/GridDhtColocatedLockFuture.java   |   1 +
 .../colocated/GridDhtDetachedCacheEntry.java    |   1 +
 .../GridDhtPartitionsAbstractMessage.java       |   1 +
 .../GridDhtPartitionsExchangeFuture.java        |   1 +
 .../preloader/GridDhtPartitionsFullMessage.java |   1 +
 .../GridDhtPartitionsSingleMessage.java         |   1 +
 .../distributed/near/GridNearAtomicCache.java   |   1 +
 .../distributed/near/GridNearCacheAdapter.java  |   1 +
 .../near/GridNearCacheClearAllRunnable.java     |   1 +
 .../distributed/near/GridNearCacheEntry.java    |   1 +
 .../distributed/near/GridNearGetFuture.java     |   1 +
 .../distributed/near/GridNearGetRequest.java    |   1 +
 .../distributed/near/GridNearGetResponse.java   |   1 +
 .../distributed/near/GridNearLockFuture.java    |   1 +
 .../distributed/near/GridNearLockRequest.java   |   1 +
 .../distributed/near/GridNearLockResponse.java  |   1 +
 .../near/GridNearTransactionalCache.java        |   1 +
 .../near/GridNearTxFinishFuture.java            |   1 +
 .../near/GridNearTxFinishRequest.java           |   2 +-
 .../near/GridNearTxFinishResponse.java          |   1 +
 .../cache/distributed/near/GridNearTxLocal.java |   1 +
 .../near/GridNearTxPrepareFuture.java           |   1 +
 .../near/GridNearTxPrepareResponse.java         |   1 +
 .../distributed/near/GridNearTxRemote.java      |   1 +
 .../cache/dr/GridCacheDrExpirationInfo.java     |   2 +-
 .../processors/cache/dr/GridCacheDrInfo.java    |   2 +-
 .../processors/cache/dr/GridCacheDrManager.java |  58 +---
 .../cache/dr/os/GridOsCacheDrManager.java       |  36 +--
 .../extras/GridCacheAttributesEntryExtras.java  |   1 +
 .../GridCacheAttributesMvccEntryExtras.java     |   1 +
 ...dCacheAttributesMvccObsoleteEntryExtras.java |   1 +
 ...cheAttributesMvccObsoleteTtlEntryExtras.java |   1 +
 .../GridCacheAttributesMvccTtlEntryExtras.java  |   1 +
 .../GridCacheAttributesObsoleteEntryExtras.java |   1 +
 ...idCacheAttributesObsoleteTtlEntryExtras.java |   1 +
 .../GridCacheAttributesTtlEntryExtras.java      |   1 +
 .../cache/extras/GridCacheEntryExtras.java      |   1 +
 .../extras/GridCacheEntryExtrasAdapter.java     |   1 +
 .../cache/extras/GridCacheMvccEntryExtras.java  |   1 +
 .../GridCacheMvccObsoleteEntryExtras.java       |   1 +
 .../GridCacheMvccObsoleteTtlEntryExtras.java    |   1 +
 .../extras/GridCacheMvccTtlEntryExtras.java     |   1 +
 .../extras/GridCacheObsoleteEntryExtras.java    |   1 +
 .../extras/GridCacheObsoleteTtlEntryExtras.java |   1 +
 .../cache/extras/GridCacheTtlEntryExtras.java   |   1 +
 .../processors/cache/local/GridLocalCache.java  |   1 +
 .../cache/local/GridLocalCacheEntry.java        |   1 +
 .../cache/local/GridLocalLockFuture.java        |   1 +
 .../cache/local/GridLocalTxFuture.java          |   1 +
 .../local/atomic/GridLocalAtomicCache.java      |   1 +
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../cache/transactions/IgniteTxAdapter.java     |  52 +++-
 .../cache/transactions/IgniteTxEntry.java       |   1 +
 .../cache/transactions/IgniteTxEx.java          |   1 +
 .../cache/transactions/IgniteTxHandler.java     |   1 +
 .../transactions/IgniteTxLocalAdapter.java      |  53 ++--
 .../cache/transactions/IgniteTxLocalEx.java     |   1 +
 .../cache/transactions/IgniteTxManager.java     |   1 +
 .../cache/transactions/IgniteTxProxyImpl.java   |   2 +-
 .../cache/transactions/IgniteTxRemoteEx.java    |   2 +-
 .../version/GridCachePlainVersionedEntry.java   | 131 +++++++++
 .../version/GridCacheRawVersionedEntry.java     | 238 ++++++++++++++++
 .../cache/version/GridCacheVersion.java         | 256 +++++++++++++++++
 ...ridCacheVersionAbstractConflictResolver.java |  56 ++++
 .../GridCacheVersionConflictContext.java        |  73 +++++
 .../GridCacheVersionConflictContextImpl.java    | 188 ++++++++++++
 .../GridCacheVersionConflictResolver.java       |  59 ++++
 .../cache/version/GridCacheVersionEx.java       | 104 +++++++
 .../cache/version/GridCacheVersionManager.java  | 282 ++++++++++++++++++
 .../cache/version/GridCacheVersionable.java     |  28 ++
 .../version/GridCacheVersionedEntryEx.java      |  31 ++
 .../dataload/IgniteDataLoaderImpl.java          |  29 +-
 .../dr/GridDrDataLoadCacheUpdater.java          |   3 +-
 .../processors/dr/GridRawVersionedEntry.java    | 210 --------------
 .../processors/dr/GridVersionedEntry.java       |  80 ------
 .../processors/fs/GridGgfsAsyncImpl.java        |   7 +-
 .../internal/processors/fs/GridGgfsImpl.java    |   2 +-
 .../handlers/cache/GridCacheCommandHandler.java |   4 +-
 .../cache/GridCacheQueryCommandHandler.java     |   4 +-
 .../handlers/task/GridTaskCommandHandler.java   |   4 +-
 .../apache/ignite/internal/util/GridUtils.java  |  15 +-
 .../GridTcpCommunicationMessageAdapter.java     |   1 +
 .../GridTcpCommunicationMessageState.java       |   3 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   2 +-
 .../ignite/lang/IgniteAsyncSupportAdapter.java  |  35 ++-
 .../ignite/lang/IgniteAsyncSupported.java       |   2 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     | 283 +++++++++++++++++++
 .../apache/ignite/IgniteCacheAffinityTest.java  | 283 -------------------
 .../internal/GridCancelOnGridStopSelfTest.java  |   2 +-
 .../GridCancelledJobsMetricsSelfTest.java       |   2 +-
 .../internal/GridContinuousTaskSelfTest.java    |   4 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   2 +-
 .../GridFailoverCustomTopologySelfTest.java     |   2 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |  34 +--
 .../internal/GridMultipleJobsSelfTest.java      |   2 +-
 .../internal/GridProjectionAbstractTest.java    |  20 +-
 .../ignite/internal/GridReduceSelfTest.java     |   2 +-
 .../GridTaskCancelSingleNodeSelfTest.java       |   2 +-
 .../internal/GridTaskExecutionSelfTest.java     |   2 +-
 .../GridTaskInstanceExecutionSelfTest.java      |   2 +-
 .../internal/GridTaskJobRejectSelfTest.java     |   2 +-
 .../GridDeploymentMessageCountSelfTest.java     |   4 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  92 +++---
 .../cache/GridCacheBasicApiAbstractTest.java    | 229 ++++++++-------
 .../GridCacheConcurrentTxMultiNodeTest.java     |   2 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |   3 +-
 .../GridCacheFinishPartitionsSelfTest.java      |  25 +-
 .../GridCacheGroupLockAbstractSelfTest.java     |  13 +-
 .../GridCacheGroupLockFailoverSelfTest.java     |   2 +-
 .../cache/GridCacheMvccFlagsTest.java           |   1 +
 .../cache/GridCacheMvccPartitionedSelfTest.java |   1 +
 .../processors/cache/GridCacheMvccSelfTest.java |   1 +
 .../cache/GridCacheNestedTxAbstractTest.java    |  13 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |  11 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   4 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   6 +
 .../cache/GridCacheVersionSelfTest.java         |   1 +
 .../cache/IgniteCacheInvokeAbstractTest.java    |   4 +-
 .../GridCacheCountDownLatchSelfTest.java        |   2 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |   4 +-
 .../GridCacheAbstractJobExecutionTest.java      |   2 +-
 .../distributed/GridCacheLockAbstractTest.java  |  62 ++--
 .../GridCacheMultiNodeLockAbstractTest.java     | 139 +++++----
 ...cOriginatingNodeFailureAbstractSelfTest.java |   2 +-
 .../dht/GridCacheColocatedDebugTest.java        |  43 +--
 .../dht/GridCacheGlobalLoadTest.java            |   2 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   1 +
 .../near/GridCacheNearMultiNodeSelfTest.java    |  25 +-
 .../near/GridCacheNearOneNodeSelfTest.java      |  42 +--
 .../near/GridCacheNearReadersSelfTest.java      |  13 +-
 .../near/GridCachePartitionedBasicApiTest.java  |   4 +-
 .../GridCachePartitionedEntryLockSelfTest.java  |   2 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |   2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   2 +-
 .../GridCacheReplicatedBasicApiTest.java        |   4 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |   7 +-
 .../local/GridCacheLocalBasicApiSelfTest.java   |   4 +-
 .../cache/local/GridCacheLocalLockSelfTest.java |  81 +++---
 .../GridCacheLocalMultithreadedSelfTest.java    |  85 +++---
 .../closure/GridClosureProcessorSelfTest.java   |  16 +-
 .../continuous/GridEventConsumeSelfTest.java    |   8 +-
 .../continuous/GridMessageListenSelfTest.java   |   4 +-
 .../processors/fs/GridGgfsTaskSelfTest.java     |   2 +-
 .../GridServiceProcessorAbstractSelfTest.java   |  18 +-
 .../GridServiceProcessorMultiNodeSelfTest.java  |   6 +-
 .../streamer/GridStreamerSelfTest.java          |   2 +-
 .../loadtest/GridSingleExecutionTest.java       |   2 +-
 .../loadtests/colocation/GridTestMain.java      |   2 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |   2 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |   2 +-
 .../ignite/loadtests/dsi/GridDsiClient.java     |   2 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   1 +
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   1 +
 ...GridJobExecutionLoadTestClientSemaphore.java |   2 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   2 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |   2 +-
 .../mergesort/GridMergeSortLoadTask.java        |   2 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   4 +-
 .../GridP2PContinuousDeploymentSelfTest.java    |   4 +-
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |  10 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |   8 +-
 .../ignite/p2p/GridP2PTimeoutSelfTest.java      |   8 +-
 .../ignite/p2p/GridP2PUndeploySelfTest.java     |   2 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |   2 +-
 ...GridSessionSetJobAttributeOrderSelfTest.java |   2 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |   2 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |   2 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |   2 +-
 .../GridSessionWaitAttributeSelfTest.java       |   2 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |  21 +-
 ...ClusterMetricsSnapshotSerializeSelfTest.java |  52 +++-
 .../cache/GridAbstractCacheStoreSelfTest.java   |   2 +-
 .../junits/common/GridCommonAbstractTest.java   |   8 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 .../hadoop/GridHadoopClassLoader.java           |   2 +-
 .../hadoop/GridHadoopPopularWordsTest.java      |   4 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |   4 +-
 .../hibernate/GridHibernateRegionFactory.java   |  10 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |   8 +-
 .../GridProjectionStartStopRestartSelfTest.java |   2 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |   2 +-
 pom.xml                                         |   4 +-
 296 files changed, 3269 insertions(+), 2290 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/641a4c87/pom.xml
----------------------------------------------------------------------


[04/12] incubator-ignite git commit: # IGNITE-102. Fixes to make benchmarks runnable.

Posted by ak...@apache.org.
# IGNITE-102. Fixes to make benchmarks runnable.


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

Branch: refs/heads/sprint-1
Commit: aee11a01f1c937a8cdef5ae88a465848be8ea533
Parents: c9aadf0
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Jan 28 15:17:37 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Jan 28 15:17:37 2015 +0700

----------------------------------------------------------------------
 .../config/benchmark-atomic-win.properties      |   2 +-
 .../config/benchmark-atomic.properties          |   2 +-
 .../config/benchmark-compute-win.properties     |   2 +-
 .../config/benchmark-compute.properties         |   2 +-
 .../config/benchmark-query-win.properties       |   2 +-
 .../yardstick/config/benchmark-query.properties |   2 +-
 .../config/benchmark-tx-win.properties          |   2 +-
 .../yardstick/config/benchmark-tx.properties    |   2 +-
 .../yardstick/config/benchmark-win.properties   |   2 +-
 modules/yardstick/config/benchmark.properties   |   2 +-
 modules/yardstick/config/ignite-base-config.xml |  10 +-
 .../config/ignite-localhost-config.xml          |   4 +-
 .../config/ignite-multicast-config.xml          |   4 +-
 modules/yardstick/pom.xml                       |   8 ++
 .../yardstick/IgniteAbstractBenchmark.java      |   6 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |  18 +--
 .../yardstick/cache/IgnitePutBenchmark.java     |  42 +++++++
 .../cache/IgnitePutBenchmarkIgnite.java         |  42 -------
 .../yardstick/cache/IgnitePutGetBenchmark.java  |  47 ++++++++
 .../cache/IgnitePutGetBenchmarkIgnite.java      |  47 --------
 .../cache/IgnitePutGetTxBenchmark.java          |  52 ++++++++
 .../cache/IgnitePutGetTxBenchmarkIgnite.java    |  52 --------
 .../yardstick/cache/IgnitePutTxBenchmark.java   |  43 +++++++
 .../cache/IgnitePutTxBenchmarkIgnite.java       |  43 -------
 .../cache/IgniteSqlQueryBenchmark.java          |  94 +++++++++++++++
 .../cache/IgniteSqlQueryBenchmarkIgnite.java    |  94 ---------------
 .../cache/IgniteSqlQueryJoinBenchmark.java      | 118 +++++++++++++++++++
 .../IgniteSqlQueryJoinBenchmarkIgnite.java      | 118 -------------------
 .../cache/IgniteSqlQueryPutBenchmark.java       |  86 ++++++++++++++
 .../cache/IgniteSqlQueryPutBenchmarkIgnite.java |  86 --------------
 30 files changed, 521 insertions(+), 513 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-atomic-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic-win.properties b/modules/yardstick/config/benchmark-atomic-win.properties
index 97132f0..96dde32 100644
--- a/modules/yardstick/config/benchmark-atomic-win.properties
+++ b/modules/yardstick/config/benchmark-atomic-win.properties
@@ -21,7 +21,7 @@ set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 :: Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 :: Probe point writer class name.
 :: BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-atomic.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-atomic.properties b/modules/yardstick/config/benchmark-atomic.properties
index d10259d..1f93d5c 100644
--- a/modules/yardstick/config/benchmark-atomic.properties
+++ b/modules/yardstick/config/benchmark-atomic.properties
@@ -23,7 +23,7 @@ JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 # Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 # Probe point writer class name.
 # BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-compute-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute-win.properties b/modules/yardstick/config/benchmark-compute-win.properties
index 06c7f6a..35e8873 100644
--- a/modules/yardstick/config/benchmark-compute-win.properties
+++ b/modules/yardstick/config/benchmark-compute-win.properties
@@ -21,7 +21,7 @@ set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 :: Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 :: Probe point writer class name.
 :: BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-compute.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-compute.properties b/modules/yardstick/config/benchmark-compute.properties
index d90a954..1f38eb6 100644
--- a/modules/yardstick/config/benchmark-compute.properties
+++ b/modules/yardstick/config/benchmark-compute.properties
@@ -23,7 +23,7 @@ JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 # Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 # Probe point writer class name.
 # BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-query-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query-win.properties b/modules/yardstick/config/benchmark-query-win.properties
index 377bac0..a2c102c 100644
--- a/modules/yardstick/config/benchmark-query-win.properties
+++ b/modules/yardstick/config/benchmark-query-win.properties
@@ -21,7 +21,7 @@ set JVM_OPTS=%JVM_OPTS%^
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 :: Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 :: Probe point writer class name.
 :: BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-query.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-query.properties b/modules/yardstick/config/benchmark-query.properties
index b467e0b..c3e7a6e 100644
--- a/modules/yardstick/config/benchmark-query.properties
+++ b/modules/yardstick/config/benchmark-query.properties
@@ -23,7 +23,7 @@ JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false" \
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 # Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 # Probe point writer class name.
 # BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-tx-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx-win.properties b/modules/yardstick/config/benchmark-tx-win.properties
index 7e098a5..7b12202 100644
--- a/modules/yardstick/config/benchmark-tx-win.properties
+++ b/modules/yardstick/config/benchmark-tx-win.properties
@@ -21,7 +21,7 @@ set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 :: Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 :: Probe point writer class name.
 :: BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-tx.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-tx.properties b/modules/yardstick/config/benchmark-tx.properties
index 83be491..c3d69f2 100644
--- a/modules/yardstick/config/benchmark-tx.properties
+++ b/modules/yardstick/config/benchmark-tx.properties
@@ -23,7 +23,7 @@ JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 # Packages where the specified benchmark is searched by reflection mechanism, comma separated.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 # Probe point writer class name.
 # BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark-win.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-win.properties b/modules/yardstick/config/benchmark-win.properties
index ba77c84..00a3c17 100644
--- a/modules/yardstick/config/benchmark-win.properties
+++ b/modules/yardstick/config/benchmark-win.properties
@@ -24,7 +24,7 @@ set JVM_OPTS=%JVM_OPTS% -DGRIDGAIN_QUIET=false
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 :: Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 :: Probe point writer class name.
 :: BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/benchmark.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark.properties b/modules/yardstick/config/benchmark.properties
index bbe9ff0..034eb46 100644
--- a/modules/yardstick/config/benchmark.properties
+++ b/modules/yardstick/config/benchmark.properties
@@ -26,7 +26,7 @@ JVM_OPTS=${JVM_OPTS}" -DGRIDGAIN_QUIET=false"
 BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
 
 # Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
 
 # Probe point writer class name.
 # BENCHMARK_WRITER=

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/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 69fe9b1..16d0a70 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -24,7 +24,7 @@
        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">
-    <bean id="base-grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" abstract="true">
+    <bean id="base-ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" abstract="true">
         <property name="peerClassLoadingEnabled" value="false"/>
 
         <!--property name="marshaller">
@@ -46,7 +46,7 @@
 
         <property name="cacheConfiguration">
             <list>
-                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                <bean class="org.apache.ignite.cache.CacheConfiguration">
                     <property name="name" value="atomic"/>
 
                     <property name="cacheMode" value="PARTITIONED"/>
@@ -58,7 +58,7 @@
                     <property name="queryIndexEnabled" value="false"/>
                 </bean>
 
-                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                <bean class="org.apache.ignite.cache.CacheConfiguration">
                     <property name="name" value="tx"/>
 
                     <property name="cacheMode" value="PARTITIONED"/>
@@ -70,7 +70,7 @@
                     <property name="queryIndexEnabled" value="false"/>
                 </bean>
 
-                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                <bean class="org.apache.ignite.cache.CacheConfiguration">
                     <property name="name" value="query"/>
 
                     <property name="cacheMode" value="PARTITIONED"/>
@@ -83,7 +83,7 @@
                     <property name="queryIndexEnabled" value="true"/>
                 </bean>
 
-                <bean class="org.gridgain.grid.cache.GridCacheConfiguration">
+                <bean class="org.apache.ignite.cache.CacheConfiguration">
                     <property name="name" value="compute"/>
 
                     <property name="cacheMode" value="PARTITIONED"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/ignite-localhost-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-localhost-config.xml b/modules/yardstick/config/ignite-localhost-config.xml
index 7a51bb0..95d881c 100644
--- a/modules/yardstick/config/ignite-localhost-config.xml
+++ b/modules/yardstick/config/ignite-localhost-config.xml
@@ -24,9 +24,9 @@
        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="gridgain-base-config.xml"/>
+    <import resource="ignite-base-config.xml"/>
 
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-grid.cfg">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-ignite.cfg">
         <property name="localHost" value="127.0.0.1"/>
 
         <property name="discoverySpi">

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/config/ignite-multicast-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-multicast-config.xml b/modules/yardstick/config/ignite-multicast-config.xml
index a81b0d5..89710c7 100644
--- a/modules/yardstick/config/ignite-multicast-config.xml
+++ b/modules/yardstick/config/ignite-multicast-config.xml
@@ -24,9 +24,9 @@
        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="gridgain-base-config.xml"/>
+    <import resource="ignite-base-config.xml"/>
 
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" parent="base-grid.cfg">
+    <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">

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 67052dc..22ce876 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -172,6 +172,14 @@
                     </filesets>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <version>2.9.1</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/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
index 5cbfee1..cca6935 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
@@ -24,8 +24,8 @@ import org.yardstickframework.*;
 
 import java.util.concurrent.*;
 
-import static org.apache.ignite.events.IgniteEventType.*;
 import static org.apache.ignite.cache.CacheDistributionMode.*;
+import static org.apache.ignite.events.IgniteEventType.*;
 import static org.yardstickframework.BenchmarkUtils.*;
 
 /**
@@ -42,7 +42,7 @@ public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
     @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
         super.setUp(cfg);
 
-        jcommander(cfg.commandLineArguments(), args, "<gridgain-driver>");
+        jcommander(cfg.commandLineArguments(), args, "<ignite-driver>");
 
         if (Ignition.state() != IgniteState.STARTED) {
             node = new IgniteNode(args.distributionMode() == CLIENT_ONLY);
@@ -79,7 +79,7 @@ public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
      * @return Grid.
      */
     protected Ignite grid() {
-        return node.grid();
+        return node.ignite();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/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
index 38a61be..b80e7bf 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -40,7 +40,7 @@ import static org.apache.ignite.cache.CacheMemoryMode.*;
  */
 public class IgniteNode implements BenchmarkServer {
     /** Grid instance. */
-    private Ignite grid;
+    private Ignite ignite;
 
     /** Client mode. */
     private boolean clientMode;
@@ -56,16 +56,16 @@ public class IgniteNode implements BenchmarkServer {
     }
 
     /** */
-    public IgniteNode(boolean clientMode, Ignite grid) {
+    public IgniteNode(boolean clientMode, Ignite ignite) {
         this.clientMode = clientMode;
-        this.grid = grid;
+        this.ignite = ignite;
     }
 
     /** {@inheritDoc} */
     @Override public void start(BenchmarkConfiguration cfg) throws Exception {
         IgniteBenchmarkArguments args = new IgniteBenchmarkArguments();
 
-        BenchmarkUtils.jcommander(cfg.commandLineArguments(), args, "<gridgain-node>");
+        BenchmarkUtils.jcommander(cfg.commandLineArguments(), args, "<ignite-node>");
 
         IgniteConfiguration c = loadConfiguration(args.configuration());
 
@@ -119,7 +119,7 @@ public class IgniteNode implements BenchmarkServer {
 
         c.setCommunicationSpi(commSpi);
 
-        grid = Ignition.start(c);
+        ignite = Ignition.start(c);
     }
 
     /**
@@ -167,7 +167,7 @@ public class IgniteNode implements BenchmarkServer {
         }
 
         if (cfgMap == null || cfgMap.isEmpty())
-            throw new Exception("Failed to find grid configuration in: " + url);
+            throw new Exception("Failed to find ignite configuration in: " + url);
 
         return cfgMap.values().iterator().next();
     }
@@ -183,9 +183,9 @@ public class IgniteNode implements BenchmarkServer {
     }
 
     /**
-     * @return Grid.
+     * @return Ignite.
      */
-    public Ignite grid() {
-        return grid;
+    public Ignite ignite() {
+        return ignite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.java
new file mode 100644
index 0000000..5136bde
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmark.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 IgnitePutBenchmark 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/aee11a01/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
deleted file mode 100644
index fbe55f3..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutBenchmarkIgnite.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.java
new file mode 100644
index 0000000..2b9fa02
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmark.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 IgnitePutGetBenchmark 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/aee11a01/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
deleted file mode 100644
index a3b3486..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetBenchmarkIgnite.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.java
new file mode 100644
index 0000000..30acb41
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmark.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 IgnitePutGetTxBenchmark 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/aee11a01/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
deleted file mode 100644
index 3cad7d7..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutGetTxBenchmarkIgnite.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.java
new file mode 100644
index 0000000..e1cefbc
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmark.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 IgnitePutTxBenchmark 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/aee11a01/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
deleted file mode 100644
index dd2fd58..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxBenchmarkIgnite.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * 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/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.java
new file mode 100644
index 0000000..38aa632
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmark.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 IgniteSqlQueryBenchmark 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/aee11a01/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
deleted file mode 100644
index c769e53..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryBenchmarkIgnite.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.java
new file mode 100644
index 0000000..7b4262b
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmark.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 IgniteSqlQueryJoinBenchmark 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/aee11a01/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
deleted file mode 100644
index 4405afa..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryJoinBenchmarkIgnite.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * 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/aee11a01/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.java
new file mode 100644
index 0000000..9d8ef4f
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmark.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 IgniteSqlQueryPutBenchmark 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/aee11a01/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
deleted file mode 100644
index 8bef1e5..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSqlQueryPutBenchmarkIgnite.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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");
-    }
-}