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

[01/14] incubator-ignite git commit: ignite-1031 Query metrics updated incorrectly

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-648 cf6d99dbb -> 18d1b8e6f


ignite-1031 Query metrics updated incorrectly


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

Branch: refs/heads/ignite-648
Commit: c04cba9e82d3f9a7a5d851047de92322c54cc568
Parents: 9d0aa6f
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 7 19:28:38 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 7 19:28:38 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 11 files changed, 361 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 5b82c34..c2425f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -36,6 +36,7 @@ import org.jetbrains.annotations.*;
 
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.*;
 
 /**
@@ -413,12 +414,19 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     @SuppressWarnings("IfMayBeConditional")
     private <R> CacheQueryFuture<R> execute(@Nullable IgniteReducer<T, R> rmtReducer,
         @Nullable IgniteClosure<T, R> rmtTransform, @Nullable Object... args) {
-        Collection<ClusterNode> nodes = nodes();
+        Collection<ClusterNode> nodes;
+
+        try {
+            nodes = nodes();
+        }
+        catch (IgniteCheckedException e) {
+            return queryErrorFuture(cctx, e, log);
+        }
 
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (nodes.isEmpty())
-            return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), new ClusterGroupEmptyCheckedException());
+            return queryErrorFuture(cctx, new ClusterGroupEmptyCheckedException(), log);
 
         if (log.isDebugEnabled())
             log.debug("Executing query [query=" + this + ", nodes=" + nodes + ']');
@@ -429,7 +437,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                 cctx.deploy().registerClasses(args);
             }
             catch (IgniteCheckedException e) {
-                return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
+                return queryErrorFuture(cctx, e, log);
             }
         }
 
@@ -457,7 +465,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     /**
      * @return Nodes to execute on.
      */
-    private Collection<ClusterNode> nodes() {
+    private Collection<ClusterNode> nodes() throws IgniteCheckedException {
         CacheMode cacheMode = cctx.config().getCacheMode();
 
         switch (cacheMode) {
@@ -466,6 +474,10 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
                     U.warn(log, "Ignoring query projection because it's executed over LOCAL cache " +
                         "(only local node will be queried): " + this);
 
+                if (type == SCAN && cctx.config().getCacheMode() == LOCAL &&
+                    partition() != null && partition() >= cctx.affinity().partitions())
+                    throw new IgniteCheckedException("Invalid partition number: " + partition());
+
                 return Collections.singletonList(cctx.localNode());
 
             case REPLICATED:
@@ -513,6 +525,21 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         });
     }
 
+    /**
+     * @param cctx Cache context.
+     * @param e Exception.
+     * @param log Logger.
+     */
+    private static <T> GridCacheQueryErrorFuture<T> queryErrorFuture(GridCacheContext<?, ?> cctx,
+        Exception e, IgniteLogger log) {
+
+        GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
+
+        GridQueryProcessor.onExecuted(cctx, metrics, null, e, 0, 0, log);
+
+        return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheQueryAdapter.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e080c6d..6af0f40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1453,6 +1453,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
             throw (IgniteCheckedException)err;
         }
+        catch (Exception e) {
+            err = e;
+
+            throw new IgniteCheckedException(e);
+        }
         finally {
             GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
index 63912bf..3eb96b6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheAbstractQueryMetricsSelfTest.java
@@ -33,7 +33,7 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
  */
 public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstractTest {
     /** Grid count. */
-    private static final int GRID_CNT = 2;
+    protected int gridCnt;
 
     /** Cache mode. */
     protected CacheMode cacheMode;
@@ -43,7 +43,7 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        startGridsMultiThreaded(GRID_CNT);
+        startGridsMultiThreaded(gridCnt);
     }
 
     /** {@inheritDoc} */
@@ -122,6 +122,57 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
+     * Test metrics for failed SQL queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlFieldsQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from UNKNOWN");
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
      * Test metrics for Scan queries.
      *
      * @throws Exception In case of error.
@@ -163,6 +214,57 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
     }
 
     /**
+     * Test metrics for failed Scan queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testScanQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        ScanQuery<String, Integer> qry = new ScanQuery<>(Integer.MAX_VALUE);
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
      * Test metrics for SQL cross cache queries.
      *
      * @throws Exception In case of error.
@@ -202,4 +304,55 @@ public abstract class CacheAbstractQueryMetricsSelfTest extends GridCommonAbstra
         assertTrue(m.maximumTime() >= 0);
         assertTrue(m.minimumTime() >= 0);
     }
+
+    /**
+     * Test metrics for failed SQL cross cache queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlCrossCacheQueryFailedMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"G\".Integer");
+
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op
+        }
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(1, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        try {
+            cache.query(qry).getAll();
+        }
+        catch (Exception e) {
+            // No-op.
+        }
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(2, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
new file mode 100644
index 0000000..d28d3a0
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheLocalQueryMetricsSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for local cache query metrics.
+ */
+public class CacheLocalQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = LOCAL;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java
new file mode 100644
index 0000000..f54a091
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsDistributedSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for partitioned distributed cache query metrics.
+ */
+public class CachePartitionedQueryMetricsDistributedSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 2;
+        cacheMode = PARTITIONED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java
new file mode 100644
index 0000000..efeb5d3
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsLocalSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for partitioned local cache query metrics.
+ */
+public class CachePartitionedQueryMetricsLocalSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = PARTITIONED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
deleted file mode 100644
index 666acfb..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CachePartitionedQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,32 +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.internal.processors.cache;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests for partitioned cache query metrics.
- */
-public class CachePartitionedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        cacheMode = PARTITIONED;
-
-        super.beforeTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java
new file mode 100644
index 0000000..ae2a420
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsDistributedSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for replicated distributed cache query metrics.
+ */
+public class CacheReplicatedQueryMetricsDistributedSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 2;
+        cacheMode = REPLICATED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java
new file mode 100644
index 0000000..d411f9c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsLocalSelfTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for replicated local cache query metrics.
+ */
+public class CacheReplicatedQueryMetricsLocalSelfTest extends CacheAbstractQueryMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        gridCnt = 1;
+        cacheMode = REPLICATED;
+
+        super.beforeTest();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
deleted file mode 100644
index 2be6dd3..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheReplicatedQueryMetricsSelfTest.java
+++ /dev/null
@@ -1,32 +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.internal.processors.cache;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests for replicated cache query metrics.
- */
-public class CacheReplicatedQueryMetricsSelfTest extends CacheAbstractQueryMetricsSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        cacheMode = REPLICATED;
-
-        super.beforeTest();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c04cba9e/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 181ff0c..2d7d0ce 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -116,8 +116,11 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(H2CompareBigQueryTest.class);
 
         // Cache query metrics.
-        suite.addTestSuite(CachePartitionedQueryMetricsSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsSelfTest.class);
+        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
 
         //Unmarshallig query test.
         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);


[10/14] incubator-ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite

Posted by sb...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-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/44b52b4a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/44b52b4a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/44b52b4a

Branch: refs/heads/ignite-648
Commit: 44b52b4a8968808ad7c17d510e56c8defe8ee837
Parents: 1cf479b 1a6948e
Author: agura <ag...@gridgain.com>
Authored: Wed Jul 8 13:31:12 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jul 8 13:31:12 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-index-put.properties       |  64 --------
 .../config/benchmark-put-indexed-val.properties |  64 ++++++++
 .../cache/IgnitePutIndex1Benchmark.java         |  42 -----
 .../cache/IgnitePutIndex2Benchmark.java         |  42 -----
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 +++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 +++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |   2 +-
 .../ignite/yardstick/cache/model/Person1.java   |  29 +---
 .../ignite/yardstick/cache/model/Person2.java   |  45 +-----
 .../ignite/yardstick/cache/model/Person8.java   | 155 ++-----------------
 10 files changed, 167 insertions(+), 360 deletions(-)
----------------------------------------------------------------------



[13/14] incubator-ignite git commit: #ignite-1101: Add test for cache future runtime exception.

Posted by sb...@apache.org.
#ignite-1101: Add test for cache future runtime exception.


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

Branch: refs/heads/ignite-648
Commit: 94a42a48530998da9895e217c99eec69db71d8c9
Parents: 44b52b4
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jul 8 14:45:49 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jul 8 14:45:49 2015 +0300

----------------------------------------------------------------------
 .../cache/CacheFutureExceptionSelfTest.java     | 121 ++++---------------
 1 file changed, 26 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/94a42a48/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
index 27bfe17..34d2daa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
@@ -19,16 +19,11 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.*;
 import java.io.*;
-import java.util.*;
 import java.util.concurrent.*;
 
 import static java.util.concurrent.TimeUnit.*;
@@ -39,9 +34,11 @@ import static java.util.concurrent.TimeUnit.*;
 public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = StartNode.createConfiguration();
+        IgniteConfiguration cfg = new IgniteConfiguration();
+        cfg.setGridName(gridName);
 
-        cfg.setClientMode(true);
+        if (gridName.equals(getTestGridName(1)))
+            cfg.setClientMode(true);
 
         return cfg;
     }
@@ -50,68 +47,39 @@ public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAsyncCacheFuture() throws Exception {
-        final CountDownLatch readyLatch = new CountDownLatch(1);
+        Ignite srv = startGrid(0);
 
-        GridJavaProcess node1 = null;
+        IgniteCache<String, NotSerializableClass> cache = srv.getOrCreateCache("CACHE");
+        cache.put("key", new NotSerializableClass());
 
-        Collection<String> jvmArgs = Arrays.asList("-ea", "-DIGNITE_QUIET=false");
+        Ignite client = startGrid(1);
 
-        try {
-            node1 = GridJavaProcess.exec(
-                StartNode.class.getName(), null,
-                log,
-                new CI1<String>() {
-                    @Override public void apply(String s) {
-                        info("Server node1: " + s);
+        IgniteCache<String, NotSerializableClass> asyncCache = client.<String, NotSerializableClass>cache("CACHE").withAsync();
 
-                        if (s.contains("Topology snapshot"))
-                            readyLatch.countDown();
-                    }
-                },
-                null,
-                jvmArgs,
-                null
-            );
+        System.setProperty("FAIL", "true");
 
-            assertTrue(readyLatch.await(60, SECONDS));
+        asyncCache.get("key");
 
-            Ignite client = startGrid(0);
+        final CountDownLatch futLatch = new CountDownLatch(1);
 
-            IgniteCache<String, NotSerializableClass> cache = client.getOrCreateCache("CACHE");
+        asyncCache.future().listen(new IgniteInClosure<IgniteFuture<Object>>() {
+            @Override public void apply(IgniteFuture<Object> fut) {
+                assertTrue(fut.isDone());
 
-            cache.put("key", new NotSerializableClass());
+                try {
+                    fut.get();
 
-            System.setProperty("FAIL", "true");
-
-            IgniteCache<String, NotSerializableClass> asyncCache = cache.withAsync();
-
-            asyncCache.get("key");
-
-            final CountDownLatch futLatch = new CountDownLatch(1);
-
-            asyncCache.future().listen(new IgniteInClosure<IgniteFuture<Object>>() {
-                @Override public void apply(IgniteFuture<Object> fut) {
-                    assertTrue(fut.isDone());
-
-                    try {
-                        fut.get();
-
-                        fail();
-                    }
-                    catch (CacheException e) {
-                        log.info("Expected error: " + e);
+                    fail();
+                }
+                catch (CacheException e) {
+                    log.info("Expected error: " + e);
 
-                        futLatch.countDown();
-                    }
+                    futLatch.countDown();
                 }
-            });
+            }
+        });
 
-            assertTrue(futLatch.await(60, SECONDS));
-        }
-        finally {
-            if (node1 != null)
-                node1.killProcess();
-        }
+        assertTrue(futLatch.await(60, SECONDS));
     }
 
     /**
@@ -131,41 +99,4 @@ public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
             in.readObject();
         }
     }
-
-    /**
-     * Test class.
-     */
-    public static class StartNode {
-        /**
-         * @return Configuration.
-         */
-        public static IgniteConfiguration createConfiguration() {
-            IgniteConfiguration cfg = new IgniteConfiguration();
-
-            cfg.setPeerClassLoadingEnabled(true);
-
-            cfg.setLocalHost("127.0.0.1");
-
-            TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-            disco.setIpFinderCleanFrequency(1000);
-
-            TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder();
-
-            ipFinder.setAddresses(Collections.singletonList("127.0.0.1:47500..47509"));
-
-            disco.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(disco);
-
-            return cfg;
-        }
-
-        /**
-         * @param args Main parameters.
-         */
-        public static void main(String[] args) {
-            Ignition.start(createConfiguration());
-        }
-    }
-}
\ No newline at end of file
+}	
\ No newline at end of file


[05/14] incubator-ignite git commit: merge from ignite-747

Posted by sb...@apache.org.
merge from ignite-747


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

Branch: refs/heads/ignite-648
Commit: 4031db76d2bd9992001a5b63f17af7739e82cff0
Parents: 0f1b31a
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 8 10:19:11 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 8 10:19:11 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 20 +++++++++++
 .../tcp/internal/TcpDiscoveryNode.java          |  2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |  8 ++++-
 .../tcp/internal/TcpDiscoveryStatistics.java    | 10 ++++--
 ...acheAtomicReplicatedNodeRestartSelfTest.java |  8 ++---
 .../tcp/TcpDiscoveryMultiThreadedTest.java      | 38 ++++++++++++++++++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |  3 ++
 7 files changed, 81 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index f8fae34..d51293e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -2881,6 +2881,24 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 msg.verify(locNodeId);
             }
+            else if (!locNodeId.equals(node.id()) && ring.node(node.id()) != null) {
+                // Local node already has node from message in local topology.
+                // Just pass it to coordinator via the ring.
+                if (ring.hasRemoteNodes())
+                    sendMessageAcrossRing(msg);
+
+                if (log.isDebugEnabled())
+                    log.debug("Local node already has node being added. Passing TcpDiscoveryNodeAddedMessage to " +
+                                  "coordinator for final processing [ring=" + ring + ", node=" + node + ", locNode="
+                                  + locNode + ", msg=" + msg + ']');
+
+                if (debugMode)
+                    debugLog("Local node already has node being added. Passing TcpDiscoveryNodeAddedMessage to " +
+                                 "coordinator for final processing [ring=" + ring + ", node=" + node + ", locNode="
+                                 + locNode + ", msg=" + msg + ']');
+
+                return;
+            }
 
             if (msg.verified() && !locNodeId.equals(node.id())) {
                 if (node.internalOrder() <= ring.maxInternalOrder()) {
@@ -3163,6 +3181,8 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (msg.verified() && locNodeId.equals(nodeId) && spiStateCopy() == CONNECTING) {
                 assert node != null;
 
+                assert topVer > 0 : "Invalid topology version: " + msg;
+
                 ring.topologyVersion(topVer);
 
                 node.order(topVer);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index 36ae39e..4b4df45 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -300,7 +300,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
      * @param order Order of the node.
      */
     public void order(long order) {
-        assert order >= 0 : "Order is invalid: " + this;
+        assert order > 0 : "Order is invalid: " + this;
 
         this.order = order;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
index e9eaa1d..acb479d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
@@ -34,7 +34,13 @@ public class TcpDiscoveryNodesRing {
     /** Visible nodes filter. */
     public static final IgnitePredicate<TcpDiscoveryNode> VISIBLE_NODES = new P1<TcpDiscoveryNode>() {
         @Override public boolean apply(TcpDiscoveryNode node) {
-            return node.visible();
+            if (node.visible()) {
+                assert node.order() > 0 : "Invalid node order: " + node;
+
+                return true;
+            }
+
+            return false;
         }
     };
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
index da8c4ea..377d8a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
@@ -256,7 +256,10 @@ public class TcpDiscoveryStatistics {
             if (maxMsgQueueTime < duration)
                 maxMsgQueueTime = duration;
 
-            avgMsgQueueTime = (avgMsgQueueTime * (totalReceivedMessages() -1)) / totalProcessedMessages();
+            int totalProcMsgs = totalProcessedMessages();
+
+            if (totalProcMsgs != 0)
+                avgMsgQueueTime = (avgMsgQueueTime * (totalProcMsgs - 1)) / totalProcMsgs;
         }
 
         msgsProcStartTs.put(msg.id(), U.currentTimeMillis());
@@ -275,7 +278,10 @@ public class TcpDiscoveryStatistics {
         if (startTs != null) {
             long duration = U.currentTimeMillis() - startTs;
 
-            avgMsgProcTime = (avgMsgProcTime * (totalProcessedMessages() - 1) + duration) / totalProcessedMessages();
+            int totalProcMsgs = totalProcessedMessages();
+
+            if (totalProcMsgs != 0)
+                avgMsgProcTime = (avgMsgProcTime * (totalProcMsgs - 1) + duration) / totalProcMsgs;
 
             if (duration > maxMsgProcTime) {
                 maxMsgProcTime = duration;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
index 54409d1..b4ed18d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
@@ -26,17 +26,17 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  */
 public class IgniteCacheAtomicReplicatedNodeRestartSelfTest extends GridCacheReplicatedNodeRestartSelfTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-747");
+    @Override public void testRestartWithPutSixNodesTwoBackups() throws Throwable {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1095");
     }
 
     /** {@inheritDoc} */
-    @Override public void testRestartWithPutSixNodesTwoBackups() throws Throwable {
+    @Override public void testRestartWithPutEightNodesTwoBackups() throws Throwable {
         fail("https://issues.apache.org/jira/browse/IGNITE-1095");
     }
 
     /** {@inheritDoc} */
-    @Override public void testRestartWithPutEightNodesTwoBackups() throws Throwable {
+    @Override public void testRestartWithPutTenNodesTwoBackups() throws Throwable {
         fail("https://issues.apache.org/jira/browse/IGNITE-1095");
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
index cfefff4..0bf7cad 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
@@ -21,8 +21,10 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.concurrent.*;
@@ -100,6 +102,8 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
      * @throws Exception If any error occurs.
      */
     public void testMultiThreaded() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1100");
+
         execute();
     }
 
@@ -126,6 +130,40 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If any error occurs.
+     */
+    public void testMultipleStartOnCoordinatorStop() throws Exception{
+        startGrids(GRID_CNT);
+
+        final CyclicBarrier barrier = new CyclicBarrier(GRID_CNT + 4);
+
+        final AtomicInteger startIdx = new AtomicInteger(GRID_CNT);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                barrier.await();
+
+                Ignite ignite = startGrid(startIdx.getAndIncrement());
+
+                assertFalse(ignite.configuration().isClientMode());
+
+                log.info("Started node: " + ignite.name());
+
+                return null;
+            }
+        }, GRID_CNT + 3, "start-thread");
+
+        barrier.await();
+
+        U.sleep(ThreadLocalRandom.current().nextInt(10, 100));
+
+        for (int i = 0; i < GRID_CNT; i++)
+            stopGrid(i);
+
+        fut.get();
+    }
+
+    /**
      * @throws Exception If failed.
      */
     private void execute() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4031db76/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index 498f50c..6f59f14 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -57,6 +57,9 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(TcpDiscoveryNodeConsistentIdSelfTest.class));
 
+        suite.addTest(new TestSuite(TcpDiscoveryRestartTest.class));
+        suite.addTest(new TestSuite(TcpDiscoveryMultiThreadedTest.class));
+
         return suite;
     }
 }


[02/14] incubator-ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite

Posted by sb...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-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/4f46ee88
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4f46ee88
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4f46ee88

Branch: refs/heads/ignite-648
Commit: 4f46ee886d4b1a40c71d0cbdab273b9eb38e3717
Parents: c04cba9 6ed5c1c
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 7 19:29:18 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jul 7 19:29:18 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/interop/InteropIgnition.java   | 17 ++++++++++++++++-
 .../ignite/internal/interop/InteropProcessor.java  |  7 +++++++
 2 files changed, 23 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[09/14] incubator-ignite git commit: Apache license added to ignite-kafka modeule

Posted by sb...@apache.org.
Apache license added to ignite-kafka modeule


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

Branch: refs/heads/ignite-648
Commit: 1cf479bca360f7381e38f7e1e8ec5e19511d0a33
Parents: 82f4992
Author: agura <ag...@gridgain.com>
Authored: Wed Jul 8 13:30:42 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jul 8 13:30:42 2015 +0300

----------------------------------------------------------------------
 modules/kafka/licenses/apache-2.0.txt | 202 +++++++++++++++++++++++++++++
 1 file changed, 202 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1cf479bc/modules/kafka/licenses/apache-2.0.txt
----------------------------------------------------------------------
diff --git a/modules/kafka/licenses/apache-2.0.txt b/modules/kafka/licenses/apache-2.0.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/modules/kafka/licenses/apache-2.0.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.


[12/14] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-648' into ignite-648

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-648' into ignite-648

Conflicts:
	modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
	modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java


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

Branch: refs/heads/ignite-648
Commit: 80ea1a3d8bbe1e7a61d90e962ab301d4a387d7c2
Parents: 00a9858 cf6d99d
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 8 14:32:21 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 8 14:32:21 2015 +0300

----------------------------------------------------------------------
 .../cache/affinity/fair/FairAffinityFunction.java  |  3 ++-
 .../ignite/internal/interop/InteropIgnition.java   | 17 ++++++++++++++++-
 .../ignite/internal/interop/InteropProcessor.java  |  7 +++++++
 .../junits/multijvm/IgniteProcessProxy.java        |  4 ++--
 4 files changed, 27 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80ea1a3d/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------


[11/14] incubator-ignite git commit: # ignite-648: review (jvm->JVM, isRemoteJvm)

Posted by sb...@apache.org.
# ignite-648: review (jvm->JVM, isRemoteJvm)


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

Branch: refs/heads/ignite-648
Commit: 00a9858300cf0e9ec7009a3c879d3ad6c6f01d7e
Parents: 83f4895
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 8 13:34:21 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 8 13:34:21 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java |  2 +-
 .../cache/GridCacheAbstractSelfTest.java        |  2 +-
 ...OnlyFairAffinityMultiJvmFullApiSelfTest.java |  2 +-
 ...AtomicClientOnlyMultiJvmFullApiSelfTest.java |  4 +-
 ...tOnlyMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...pyOnReadDisabledMultiJvmFullApiSelfTest.java |  4 +-
 ...omicFairAffinityMultiJvmFullApiSelfTest.java |  4 +-
 .../GridCacheAtomicMultiJvmFullApiSelfTest.java |  2 +-
 ...tomicMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...bledFairAffinityMultiJvmFullApiSelfTest.java |  4 +-
 ...tomicNearEnabledMultiJvmFullApiSelfTest.java |  4 +-
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |  4 +-
 ...heAtomicNearOnlyMultiJvmFullApiSelfTest.java |  2 +-
 ...rOnlyMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...cheAtomicOffHeapMultiJvmFullApiSelfTest.java |  4 +-
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |  2 +-
 ...rderFairAffinityMultiJvmFullApiSelfTest.java |  4 +-
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |  4 +-
 ...OrderMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...rityOrderOffHeapMultiJvmFullApiSelfTest.java |  4 +-
 ...derOffHeapTieredMultiJvmFullApiSelfTest.java |  2 +-
 ...OnlyFairAffinityMultiJvmFullApiSelfTest.java |  4 +-
 ...ridCacheNearOnlyMultiJvmFullApiSelfTest.java |  2 +-
 ...rOnlyMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 .../GridCacheNearTxMultiJvmSelfTest.java        |  2 +-
 ...pyOnReadDisabledMultiJvmFullApiSelfTest.java |  4 +-
 ...onedFairAffinityMultiJvmFullApiSelfTest.java |  4 +-
 ...CachePartitionedMultiJvmCounterSelfTest.java |  4 +-
 ...CachePartitionedMultiJvmFullApiSelfTest.java |  2 +-
 ...ionedMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...micOffHeapTieredMultiJvmFullApiSelfTest.java |  2 +-
 ...bledFairAffinityMultiJvmFullApiSelfTest.java |  4 +-
 ...onedNearDisabledMultiJvmFullApiSelfTest.java |  4 +-
 ...abledMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...rDisabledOffHeapMultiJvmFullApiSelfTest.java |  4 +-
 ...ledOffHeapTieredMultiJvmFullApiSelfTest.java |  2 +-
 ...rtitionedOffHeapMultiJvmFullApiSelfTest.java |  4 +-
 ...nedOffHeapTieredMultiJvmFullApiSelfTest.java |  2 +-
 ...ReplicatedAtomicMultiJvmFullApiSelfTest.java |  2 +-
 ...rimaryWriteOrderMultiJvmFullApiSelfTest.java |  4 +-
 ...dCacheReplicatedMultiJvmFullApiSelfTest.java |  2 +-
 ...catedMultiJvmP2PDisabledFullApiSelfTest.java |  4 +-
 ...plicatedNearOnlyMultiJvmFullApiSelfTest.java |  2 +-
 ...eplicatedOffHeapMultiJvmFullApiSelfTest.java |  4 +-
 ...tedOffHeapTieredMultiJvmFullApiSelfTest.java |  2 +-
 .../testframework/junits/GridAbstractTest.java  | 46 +++++++++++---------
 .../junits/multijvm/AffinityProcessProxy.java   |  2 +-
 .../multijvm/IgniteClusterProcessProxy.java     |  2 +-
 .../junits/multijvm/IgniteNodeRunner.java       |  2 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  6 +--
 ...IgniteCacheFullApiMultiJvmSelfTestSuite.java | 12 +----
 51 files changed, 106 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 65681dc..238332a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -207,7 +207,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         cfg.setCacheConfiguration();
 
-        if (!isMultiJvmAndNodeIsRemote(gridName))
+        if (!isRemoteJvm(gridName))
             return IgnitionEx.start(optimize(cfg), ctx);
         else
             return startRemoteGrid(gridName, optimize(cfg), ctx);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index d98a550..4cd208d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -369,7 +369,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      * @return Cache context.
      */
     protected GridCacheContext<String, Integer> context(final int idx) {
-        if (isMultiJvmAndNodeIsRemote(idx) && !weAreOnRemoteJvm())
+        if (isRemoteJvm(idx) && !isRemoteJvm())
             throw new UnsupportedOperationException("Operation can't be done automatically via proxy. " +
                 "Send task with this logic on remote jvm instead.");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
index 85269f5..e2ac8a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest
     extends GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest.java
index 0768b9f..dfffc70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest extends
     GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest.java
index 3c33730..2d8b9a5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest extends
     GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest.java
index 781b14b..1388a75 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest extends
     GridCacheAtomicCopyOnReadDisabledMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
index 164f14b..5808f28 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest extends
     GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmFullApiSelfTest.java
index 6064d05..0d6a1ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheAtomicMultiJvmFullApiSelfTest extends GridCacheAtomicMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest.java
index fb5348b..0f35862 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest extends
     GridCacheAtomicMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
index 3d2debc..44e0a6d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest extends
     GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest.java
index 0177d2b..44f3c6f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest extends
     GridCacheAtomicNearEnabledMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest.java
index 3fcd505..e23ab52 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest extends
     GridCacheAtomicNearEnabledPrimaryWriteOrderMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
index ba01611..a2850c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest extends
     GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
index 8b67346..f07d51d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest extends
     GridCacheAtomicNearOnlyMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapMultiJvmFullApiSelfTest.java
index 01fbaab..8eb4db1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicOffHeapMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicOffHeapMultiJvmFullApiSelfTest extends
     GridCacheAtomicOffHeapMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapTieredMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
index 7b2aad1..acc1892 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheAtomicOffHeapTieredMultiJvmFullApiSelfTest
     extends GridCacheAtomicOffHeapTieredMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
index 2838d02..421ce32 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest extends
     GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
index 82498ce..928b955 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest extends
     GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest.java
index 2659d11..c805377 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest extends
     GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest.java
index 5f4a5c1..3732cd2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest extends 
+public class GridCacheAtomicPrimaryWrityOrderOffHeapMultiJvmFullApiSelfTest extends
     GridCacheAtomicPrimaryWrityOrderOffHeapMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiJvmFullApiSelfTest.java
index ed43c15..192b4a5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiJvmFullApiSelfTest
     extends GridCacheAtomicPrimaryWrityOrderOffHeapTieredMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
index e4fd892..c8829a1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest extends 
+public class GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest extends
     GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmFullApiSelfTest.java
index fdc3ebb..40504d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheNearOnlyMultiJvmFullApiSelfTest extends GridCacheNearOnlyMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
index 9a83d14..5e48504 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest extends
     GridCacheNearOnlyMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearTxMultiJvmSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearTxMultiJvmSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearTxMultiJvmSelfTest.java
index a74f7cd..467e32c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearTxMultiJvmSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearTxMultiJvmSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheNearTxMultiJvmSelfTest extends GridCacheNearTxMultiNodeSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest.java
index eaa5f79..2aa8ed8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest extends 
+public class GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest extends
     GridCachePartitionedCopyOnReadDisabledMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
index 48335bb..7bd0043 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest extends 
+public class GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest extends
     GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmCounterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmCounterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmCounterSelfTest.java
index 7f39342..8644ca6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmCounterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmCounterSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedMultiJvmCounterSelfTest extends 
+public class GridCachePartitionedMultiJvmCounterSelfTest extends
     GridCachePartitionedMultiNodeCounterSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmFullApiSelfTest.java
index 336949a..c88f360 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCachePartitionedMultiJvmFullApiSelfTest extends GridCachePartitionedMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest.java
index dc3a060..1e619eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest extends
     GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
index 42bf885..c4332df 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest
     extends GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
index 00459d6..b3e45ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest extends 
+public class GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest extends
     GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest.java
index ba37f99..6cdd3be 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest extends 
+public class GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest extends
     GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest.java
index 7cabc39..c873b11 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest extends
     GridCachePartitionedNearDisabledMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest.java
index bebd3ad..8d0bc7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest extends 
+public class GridCachePartitionedNearDisabledOffHeapMultiJvmFullApiSelfTest extends
     GridCachePartitionedNearDisabledOffHeapMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest.java
index 519d840..486ca2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest
     extends GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapMultiJvmFullApiSelfTest.java
index 6d10111..5cf5d8a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCachePartitionedOffHeapMultiJvmFullApiSelfTest extends 
+public class GridCachePartitionedOffHeapMultiJvmFullApiSelfTest extends
     GridCachePartitionedOffHeapMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapTieredMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapTieredMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapTieredMultiJvmFullApiSelfTest.java
index 6e44dd1..d2d45b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapTieredMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedOffHeapTieredMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCachePartitionedOffHeapTieredMultiJvmFullApiSelfTest
     extends GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicMultiJvmFullApiSelfTest.java
index a4a50a4..7f8bfa1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheReplicatedAtomicMultiJvmFullApiSelfTest extends
     GridCacheReplicatedAtomicMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
index 0f7a180..fb87c9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest extends 
+public class GridCacheReplicatedAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest extends
     GridCacheReplicatedAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
index b135099..19a5abd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheReplicatedMultiJvmFullApiSelfTest extends GridCacheReplicatedMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest.java
index 96de19b..7473bdc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest extends 
+public class GridCacheReplicatedMultiJvmP2PDisabledFullApiSelfTest extends
     GridCacheReplicatedMultiNodeP2PDisabledFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedNearOnlyMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedNearOnlyMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedNearOnlyMultiJvmFullApiSelfTest.java
index 90f2eb8..9d7639c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedNearOnlyMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedNearOnlyMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheReplicatedNearOnlyMultiJvmFullApiSelfTest extends
     GridCacheReplicatedNearOnlyMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest.java
index 5bd9dd6..f054f4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
-public class GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest extends 
+public class GridCacheReplicatedOffHeapMultiJvmFullApiSelfTest extends
     GridCacheReplicatedOffHeapMultiNodeFullApiSelfTest {
     /** {@inheritDoc} */
     protected boolean isMultiJvm() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapTieredMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapTieredMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapTieredMultiJvmFullApiSelfTest.java
index d859c58..dcf2721 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapTieredMultiJvmFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheReplicatedOffHeapTieredMultiJvmFullApiSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.multijvm;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
- * Multy Jvm tests.
+ * Multi-JVM tests.
  */
 public class GridCacheReplicatedOffHeapTieredMultiJvmFullApiSelfTest
     extends GridCacheReplicatedOffHeapTieredMultiNodeFullApiSelfTest {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 3cdced7..3d27d2d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -211,7 +211,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @return logger.
      */
     protected IgniteLogger log() {
-        if (weAreOnRemoteJvm())
+        if (isRemoteJvm())
             return IgniteNodeRunner.startedInstance().log();
 
         return log;
@@ -676,7 +676,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
-        if (!isMultiJvmAndNodeIsRemote(gridName)) {
+        if (!isRemoteJvm(gridName)) {
             startingGrid.set(gridName);
 
             try {
@@ -755,7 +755,7 @@ public abstract class GridAbstractTest extends TestCase {
 
             info(">>> Stopping grid [name=" + ignite.name() + ", id=" + ignite.cluster().localNode().id() + ']');
 
-            if (!isMultiJvmAndNodeIsRemote(gridName))
+            if (!isRemoteJvm(gridName))
                 G.stop(gridName, cancel);
             else
                 IgniteProcessProxy.stop(gridName, cancel);
@@ -801,7 +801,7 @@ public abstract class GridAbstractTest extends TestCase {
             assert G.allGrids().isEmpty();
         }
         finally {
-            IgniteProcessProxy.killAll(); // In multi jvm case.
+            IgniteProcessProxy.killAll(); // In multi-JVM case.
         }
     }
 
@@ -871,10 +871,10 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Grid instance.
      */
     protected IgniteEx grid(String name) {
-        if (!isMultiJvmAndNodeIsRemote(name))
+        if (!isRemoteJvm(name))
             return (IgniteEx)G.ignite(name);
         else {
-            if (weAreOnRemoteJvm())
+            if (isRemoteJvm())
                 return IgniteNodeRunner.startedInstance();
             else
                 return IgniteProcessProxy.ignite(name);
@@ -956,7 +956,7 @@ public abstract class GridAbstractTest extends TestCase {
     protected Ignite startGrid(String gridName, IgniteConfiguration cfg) throws Exception {
         cfg.setGridName(gridName);
 
-        if (!isMultiJvmAndNodeIsRemote(gridName))
+        if (!isRemoteJvm(gridName))
             return G.start(cfg);
         else
             return startRemoteGrid(gridName, cfg, null);
@@ -1362,11 +1362,14 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Gets flag whether nodes will run in one jvm or in separate jvms.
+     * Gets flag whether nodes will run in one JVM or in separate JVMs.
      *
-     * @return <code>True</code> to run nodes in separate jvms.
+     * @return <code>True</code> to run nodes in separate JVMs.
      * @see IgniteNodeRunner
      * @see IgniteProcessProxy
+     * @see #isRemoteJvm()
+     * @see #isRemoteJvm(int)
+     * @see #isRemoteJvm(String)
      * @see #executeOnLocalOrRemoteJvm(int, TestIgniteIdxCallable)
      * @see #executeOnLocalOrRemoteJvm(Ignite, TestIgniteCallable)
      * @see #executeOnLocalOrRemoteJvm(IgniteCache, TestCacheCallable)
@@ -1377,24 +1380,25 @@ public abstract class GridAbstractTest extends TestCase {
 
     /**
      * @param gridName Grid name.
-     * @return <code>True</code> if test was run in multy jvm mode and grid at another jvm.
+     * @return <code>True</code> if test was run in multi-JVM mode and grid with this name was started at another JVM.
      */
-    protected boolean isMultiJvmAndNodeIsRemote(String gridName) {
+    protected boolean isRemoteJvm(String gridName) {
         return isMultiJvm() && !gridName.endsWith("0");
     }
 
     /**
      * @param idx Grid index.
-     * @return <code>True</code> if test was run in multy jvm mode and grid at another jvm.
+     * @return <code>True</code> if test was run in multi-JVM mode and grid with this ID was started at another JVM.
      */
-    protected boolean isMultiJvmAndNodeIsRemote(int idx) {
+    protected boolean isRemoteJvm(int idx) {
         return isMultiJvm() && idx != 0;
     }
 
     /**
-     * @return <code>True</code> if current jvm contains remote started node.
+     * @return <code>True</code> if current JVM contains remote started node
+     * (It differs from JVM where tests executing).
      */
-    protected boolean weAreOnRemoteJvm() {
+    protected boolean isRemoteJvm() {
         return IgniteNodeRunner.hasStartedInstance();
     }
 
@@ -1415,7 +1419,7 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Calls job on local jvm or on remote jvm in multi jvm case.
+     * Calls job on local JVM or on remote JVM in multi-JVM case.
      *
      * @param idx Grid index.
      * @param job Job.
@@ -1435,7 +1439,7 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Calls job on local jvm or on remote jvm in multi jvm case.
+     * Calls job on local JVM or on remote JVM in multi-JVM case.
      *
      * @param ignite Ignite.
      * @param job Job.
@@ -1453,7 +1457,7 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Calls job on local jvm or on remote jvm in multi jvm case.
+     * Calls job on local JVM or on remote JVM in multi-JVM case.
      *
      * @param cache Cache.
      * @param job Job.
@@ -1473,7 +1477,7 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Calls job on remote jvm.
+     * Calls job on remote JVM.
      *
      * @param idx Grid index.
      * @param job Job.
@@ -1494,7 +1498,7 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Calls job on remote jvm.
+     * Calls job on remote JVM.
      *
      * @param proxy Ignite.
      * @param job Job.
@@ -1512,7 +1516,7 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * Runs job on remote jvm.
+     * Runs job on remote JVM.
      *
      * @param cache Cache.
      * @param job Job.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
index 1c816f5..e87895d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/AffinityProcessProxy.java
@@ -26,7 +26,7 @@ import org.jetbrains.annotations.*;
 import java.util.*;
 
 /**
- * Proxy class for affinity at another jvm.
+ * Proxy class for affinity at another JVM.
  */
 @SuppressWarnings("TransientFieldInNonSerializableClass")
 public class AffinityProcessProxy<K> implements Affinity<K> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
index 99ec424..159c451 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
@@ -28,7 +28,7 @@ import java.util.*;
 import java.util.concurrent.*;
 
 /**
- * Proxy class for cluster at another jvm.
+ * Proxy class for cluster at another JVM.
  */
 @SuppressWarnings("TransientFieldInNonSerializableClass")
 public class IgniteClusterProcessProxy implements IgniteClusterEx {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
index 04721e8..22411e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteNodeRunner.java
@@ -67,7 +67,7 @@ public class IgniteNodeRunner {
     }
 
     /**
-     * @return <code>True</code> if there is ignite node started via {@link IgniteNodeRunner} at this jvm.
+     * @return <code>True</code> if there is ignite node started via {@link IgniteNodeRunner} at this JVM.
      */
     public static boolean hasStartedInstance() {
         return ignite != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 7ac652a..a3161be 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -51,7 +51,7 @@ public class IgniteProcessProxy implements IgniteEx {
     /** Configuration. */
     private final transient IgniteConfiguration cfg;
 
-    /** Local jvm grid. */
+    /** Local JVM grid. */
     private final transient Ignite locJvmGrid;
 
     /** Logger. */
@@ -63,7 +63,7 @@ public class IgniteProcessProxy implements IgniteEx {
     /**
      * @param cfg Configuration.
      * @param log Logger.
-     * @param locJvmGrid Local jvm grid.
+     * @param locJvmGrid Local JVM grid.
      */
     public IgniteProcessProxy(final IgniteConfiguration cfg, final IgniteLogger log, final Ignite locJvmGrid)
         throws Exception {
@@ -101,7 +101,7 @@ public class IgniteProcessProxy implements IgniteEx {
             System.getProperty("surefire.test.class.path")
         );
 
-        assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node with id=" + id + " didn't join.";
+        assert rmtNodeStartedLatch.await(30, TimeUnit.SECONDS): "Remote node is not joined [id=" + id + "]";
 
         gridProxies.put(cfg.getGridName(), this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/00a98583/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
index 75d34c4..f52c031 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
@@ -21,11 +21,11 @@ import junit.framework.*;
 import org.apache.ignite.internal.processors.cache.multijvm.*;
 
 /**
- * Multi jvm test suite.
+ * Multi-JVM test suite.
  */
 public class IgniteCacheFullApiMultiJvmSelfTestSuite extends TestSuite {
     /**
-     * @return Multi JVM tests suite.
+     * @return Multi-JVM tests suite.
      * @throws Exception If failed.
      */
     public static TestSuite suite() throws Exception {
@@ -69,10 +69,6 @@ public class IgniteCacheFullApiMultiJvmSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.class);
 
-//        suite.addTestSuite(GridCacheNearReloadAllSelfTest.class);
-//        suite.addTestSuite(GridCacheColocatedReloadAllSelfTest.class);
-//        suite.addTestSuite(GridCacheAtomicReloadAllSelfTest.class);
-//        suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderReloadAllSelfTest.class);
         suite.addTestSuite(GridCacheNearTxMultiJvmSelfTest.class);
         suite.addTestSuite(GridCachePartitionedMultiJvmCounterSelfTest.class);
 
@@ -91,10 +87,6 @@ public class IgniteCacheFullApiMultiJvmSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedNearDisabledOffHeapTieredMultiJvmFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiJvmFullApiSelfTest.class);
 
-        // Multithreaded
-//        suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);
-//        suite.addTestSuite(GridCachePartitionedFullApiMultithreadedSelfTest.class);
-
         return suite;
     }
 }


[06/14] incubator-ignite git commit: Added benchmarks.

Posted by sb...@apache.org.
Added benchmarks.


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

Branch: refs/heads/ignite-648
Commit: 946b08ba435813881920f8547d7cff36a9faf9a2
Parents: 4031db7
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jul 8 12:30:41 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jul 8 12:30:41 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-index-put.properties       |  64 +++++
 modules/yardstick/config/ignite-base-config.xml |  23 ++
 .../cache/IgnitePutIndex1Benchmark.java         |  42 ++++
 .../cache/IgnitePutIndex2Benchmark.java         |  42 ++++
 .../cache/IgnitePutIndex8Benchmark.java         |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  78 ++++++
 .../ignite/yardstick/cache/model/Person2.java   | 104 ++++++++
 .../ignite/yardstick/cache/model/Person8.java   | 242 +++++++++++++++++++
 8 files changed, 637 insertions(+)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/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 9d28774..b2c976a 100644
--- a/modules/yardstick/config/ignite-base-config.xml
+++ b/modules/yardstick/config/ignite-base-config.xml
@@ -104,6 +104,29 @@
                 </bean>
 
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="atomic-index"/>
+
+                    <property name="cacheMode" value="PARTITIONED"/>
+
+                    <property name="atomicityMode" value="ATOMIC"/>
+
+                    <property name="swapEnabled" value="false"/>
+
+                    <property name="indexedTypes">
+                        <list>
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person1</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person2</value>
+
+                            <value>java.lang.Integer</value>
+                            <value>org.apache.ignite.yardstick.cache.model.Person8</value>
+                        </list>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
                     <property name="name" value="query"/>
 
                     <property name="cacheMode" value="PARTITIONED"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.java
new file mode 100644
index 0000000..65dd39a
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.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.*;
+
+/**
+ * Ignite benchmark that performs put operations for entity with indexed fields.
+ */
+public class IgnitePutIndex1Benchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new Person1(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.java
new file mode 100644
index 0000000..b4cf86f
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.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.*;
+
+/**
+ * Ignite benchmark that performs put operations for entity with indexed fields.
+ */
+public class IgnitePutIndex2Benchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new Person2(key, key + 1));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.java
new file mode 100644
index 0000000..542a139
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.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.*;
+
+/**
+ * Ignite benchmark that performs put operations for entity with indexed fields.
+ */
+public class IgnitePutIndex8Benchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new Person8(key, key + 1, key + 2, key + 3, key + 4, key + 5, key + 6, key + 7));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
new file mode 100644
index 0000000..0b2a0bf
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
@@ -0,0 +1,78 @@
+/*
+ * 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.annotations.*;
+
+import java.io.*;
+
+/**
+ * Value used for indexed put test.
+ */
+public class Person1 implements Serializable {
+    /** Value. */
+    @QuerySqlField(index = true)
+    private int val1;
+
+    /**
+     * Empty constructor.
+     */
+    public Person1() {
+        // No-op.
+    }
+
+    /**
+     * Constructs.
+     *
+     * @param val1 Indexed value.
+     */
+    public Person1(int val1) {
+        this.val1 = val1;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal1() {
+        return val1;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val1 Value.
+     */
+    public void setVal1(int val1) {
+        this.val1 = val1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        return this == o || (o instanceof Person1) && val1 == ((Person1)o).val1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return val1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "IndexedValue1 [val1=" + val1 + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
new file mode 100644
index 0000000..3b1cb9e
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
@@ -0,0 +1,104 @@
+/*
+ * 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.annotations.*;
+
+import java.io.*;
+
+/**
+ * Value used for indexed put test.
+ */
+public class Person2 implements Serializable {
+    /** Value 1. */
+    @QuerySqlField(index = true)
+    private int val1;
+
+    /** Value 2. */
+    @QuerySqlField(index = true)
+    private int val2;
+
+    /**
+     * Empty constructor.
+     */
+    public Person2() {
+        // No-op.
+    }
+
+    /**
+     * Constructs.
+     */
+    public Person2(int val1, int val2) {
+        this.val1 = val1;
+        this.val2 = val2;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal1() {
+        return val1;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val1 Value.
+     */
+    public void setVal1(int val1) {
+        this.val1 = val1;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal2() {
+        return val2;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val2 Value.
+     */
+    public void setVal2(int val2) {
+        this.val2 = val2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Person2 value2 = (Person2)o;
+
+        return val1 == value2.val1 && val2 == value2.val2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return 31 * val1 + val2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "IndexedValue2 [val1=" + val1 + ", val2=" + val2 + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
new file mode 100644
index 0000000..b8e3f7c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
@@ -0,0 +1,242 @@
+/*
+ * 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.annotations.*;
+
+import java.io.*;
+
+/**
+ * Value used for indexed put test.
+ */
+public class Person8 implements Serializable {
+    /** Value 1. */
+    @QuerySqlField(index = true)
+    private int val1;
+
+    /** Value 2. */
+    @QuerySqlField(index = true)
+    private int val2;
+
+    /** Value 3. */
+    @QuerySqlField(index = true)
+    private int val3;
+
+    /** Value 4. */
+    @QuerySqlField(index = true)
+    private int val4;
+
+    /** Value 5. */
+    @QuerySqlField(index = true)
+    private int val5;
+
+    /** Value 6. */
+    @QuerySqlField(index = true)
+    private int val6;
+
+    /** Value 7. */
+    @QuerySqlField(index = true)
+    private int val7;
+
+    /** Value 8. */
+    @QuerySqlField(index = true)
+    private int val8;
+
+    /**
+     * Empty constructor.
+     */
+    public Person8() {
+        // No-op.
+    }
+
+    /**
+     * Constructs.
+     */
+    public Person8(int val1, int val2, int val3, int val4, int val5, int val6, int val7, int val8) {
+        this.val1 = val1;
+        this.val2 = val2;
+        this.val3 = val3;
+        this.val4 = val4;
+        this.val5 = val5;
+        this.val6 = val6;
+        this.val7 = val7;
+        this.val8 = val8;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal1() {
+        return val1;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val1 Value.
+     */
+    public void setVal1(int val1) {
+        this.val1 = val1;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal2() {
+        return val2;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val2 Value.
+     */
+    public void setVal2(int val2) {
+        this.val2 = val2;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal3() {
+        return val3;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val3 Value.
+     */
+    public void setVal3(int val3) {
+        this.val3 = val3;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal4() {
+        return val4;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val4 Value.
+     */
+    public void setVal4(int val4) {
+        this.val4 = val4;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal5() {
+        return val5;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val5 Value.
+     */
+    public void setVal5(int val5) {
+        this.val5 = val5;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal6() {
+        return val6;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val6 Value.
+     */
+    public void setVal6(int val6) {
+        this.val6 = val6;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal7() {
+        return val7;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val7 Value.
+     */
+    public void setVal7(int val7) {
+        this.val7 = val7;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal8() {
+        return val8;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val8 Value.
+     */
+    public void setVal8(int val8) {
+        this.val8 = val8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Person8 p8 = (Person8)o;
+
+        return val1 == p8.val1 && val2 == p8.val2 && val3 == p8.val3 && val4 == p8.val4
+            && val5 == p8.val5 && val6 == p8.val6 && val7 == p8.val7 && val8 == p8.val8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = val1;
+
+        result = 31 * result + val2;
+        result = 31 * result + val3;
+        result = 31 * result + val4;
+        result = 31 * result + val5;
+        result = 31 * result + val6;
+        result = 31 * result + val7;
+        result = 31 * result + val8;
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Person8 [val1=" + val1 + ", val2=" + val2 + ", val3=" + val3 + ", val4=" + val4 + ", val5=" + val5 +
+            ", val6=" + val6 + ", val7=" + val7 + ", val8=" + val8 +']';
+    }
+}


[04/14] incubator-ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-648
Commit: 0f1b31a5b89f7406bfe44e488a9a832567ee1ccb
Parents: 17ee8a6 4f46ee8
Author: ashutak <as...@gridgain.com>
Authored: Tue Jul 7 20:39:42 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jul 7 20:39:42 2015 +0300

----------------------------------------------------------------------
 .../internal/interop/InteropIgnition.java       |  17 +-
 .../internal/interop/InteropProcessor.java      |   7 +
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 13 files changed, 384 insertions(+), 73 deletions(-)
----------------------------------------------------------------------



[14/14] incubator-ignite git commit: Merge branch 'master' into ignite-648

Posted by sb...@apache.org.
Merge branch 'master' into ignite-648


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

Branch: refs/heads/ignite-648
Commit: 18d1b8e6fe4a5be363e01b7bcbdea1c6f3bedb94
Parents: 80ea1a3 94a42a4
Author: ashutak <as...@gridgain.com>
Authored: Wed Jul 8 15:07:02 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Wed Jul 8 15:07:02 2015 +0300

----------------------------------------------------------------------
 dev-tools/slurp.sh                              |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |  35 +++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  20 ++
 .../tcp/internal/TcpDiscoveryNode.java          |   2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   8 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |  10 +-
 .../cache/CacheFutureExceptionSelfTest.java     | 121 +++--------
 ...acheAtomicReplicatedNodeRestartSelfTest.java |   8 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  38 ++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   3 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 +++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 +++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 +++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 +++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ---
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 +++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 +++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ---
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 modules/yardstick/config/ignite-base-config.xml |  23 +++
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  55 +++++
 .../ignite/yardstick/cache/model/Person2.java   |  67 ++++++
 .../ignite/yardstick/cache/model/Person8.java   | 109 ++++++++++
 scripts/git-patch-prop.sh                       |   2 +-
 30 files changed, 1116 insertions(+), 177 deletions(-)
----------------------------------------------------------------------



[07/14] incubator-ignite git commit: # review

Posted by sb...@apache.org.
# review


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

Branch: refs/heads/ignite-648
Commit: 82f49921969979589e04fb12f9902201a29ac7b2
Parents: 946b08b
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 12:38:11 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 12:38:11 2015 +0300

----------------------------------------------------------------------
 .../cache/IgnitePutIndex8Benchmark.java         | 42 --------------------
 .../cache/IgnitePutIndexedValue8Benchmark.java  | 42 ++++++++++++++++++++
 2 files changed, 42 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/82f49921/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.java
deleted file mode 100644
index 542a139..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex8Benchmark.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.*;
-
-/**
- * Ignite benchmark that performs put operations for entity with indexed fields.
- */
-public class IgnitePutIndex8Benchmark extends IgniteCacheAbstractBenchmark {
-    /** {@inheritDoc} */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        int key = nextRandom(args.range());
-
-        cache.put(key, new Person8(key, key + 1, key + 2, key + 3, key + 4, key + 5, key + 6, key + 7));
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteCache<Integer, Object> cache() {
-        return ignite().cache("atomic-index");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/82f49921/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java
new file mode 100644
index 0000000..b51ff34
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.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.*;
+
+/**
+ * Ignite benchmark that performs put operations for entity with indexed fields.
+ */
+public class IgnitePutIndexedValue8Benchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new Person8(key, key + 1, key + 2, key + 3, key + 4, key + 5, key + 6, key + 7));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}


[03/14] incubator-ignite git commit: # master: update DEFAULT_BRANCH to master.

Posted by sb...@apache.org.
# master: update DEFAULT_BRANCH to master.


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

Branch: refs/heads/ignite-648
Commit: 17ee8a6b837c6266b5065b956df0918305ec8814
Parents: 9d0aa6f
Author: ashutak <as...@gridgain.com>
Authored: Tue Jul 7 20:39:14 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jul 7 20:39:14 2015 +0300

----------------------------------------------------------------------
 dev-tools/slurp.sh        | 2 +-
 scripts/git-patch-prop.sh | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ee8a6b/dev-tools/slurp.sh
----------------------------------------------------------------------
diff --git a/dev-tools/slurp.sh b/dev-tools/slurp.sh
index 7edc776..ad7bc01 100755
--- a/dev-tools/slurp.sh
+++ b/dev-tools/slurp.sh
@@ -24,7 +24,7 @@
 #
 # Default branch name. Need to use last version of "jiraslurp" scripts.
 #
-DEFAULT_BRANCH='ignite-sprint-5'
+DEFAULT_BRANCH='master'
 
 #
 # TC URL. It is 10.30.0.229 for public TC from agents.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17ee8a6b/scripts/git-patch-prop.sh
----------------------------------------------------------------------
diff --git a/scripts/git-patch-prop.sh b/scripts/git-patch-prop.sh
index 16f907e..1b531fa 100644
--- a/scripts/git-patch-prop.sh
+++ b/scripts/git-patch-prop.sh
@@ -19,6 +19,6 @@
 #
 # Git patch-file maker/applier properties.
 #
-IGNITE_DEFAULT_BRANCH='ignite-sprint-7'
+IGNITE_DEFAULT_BRANCH='master'
 
 PATCHES_HOME=${IGNITE_HOME}


[08/14] incubator-ignite git commit: Fixed notes

Posted by sb...@apache.org.
Fixed notes


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

Branch: refs/heads/ignite-648
Commit: 1a6948e8778d0e6379ffc0c99fee0d83d8bbebd7
Parents: 82f4992
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jul 8 13:04:31 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jul 8 13:04:31 2015 +0300

----------------------------------------------------------------------
 .../config/benchmark-index-put.properties       |  64 --------
 .../config/benchmark-put-indexed-val.properties |  64 ++++++++
 .../cache/IgnitePutIndex1Benchmark.java         |  42 -----
 .../cache/IgnitePutIndex2Benchmark.java         |  42 -----
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 +++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 +++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |   2 +-
 .../ignite/yardstick/cache/model/Person1.java   |  29 +---
 .../ignite/yardstick/cache/model/Person2.java   |  45 +-----
 .../ignite/yardstick/cache/model/Person8.java   | 155 ++-----------------
 10 files changed, 167 insertions(+), 360 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/config/benchmark-index-put.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-index-put.properties b/modules/yardstick/config/benchmark-index-put.properties
deleted file mode 100644
index fb73656..0000000
--- a/modules/yardstick/config/benchmark-index-put.properties
+++ /dev/null
@@ -1,64 +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.
-
-#
-# Contains benchmarks for SQL queries.
-#
-
-# JVM options.
-# JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
-
-# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false" \
-  -XX:+UseParNewGC \
-  -XX:+UseConcMarkSweepGC \
-  -XX:+UseTLAB \
-  -XX:NewSize=128m \
-  -XX:MaxNewSize=128m \
-  -XX:MaxTenuringThreshold=0 \
-  -XX:SurvivorRatio=1024 \
-  -XX:+UseCMSInitiatingOccupancyOnly \
-  -XX:CMSInitiatingOccupancyFraction=60 \
-"
-
-# List of default probes.
-# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
-BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
-
-# Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
-
-# Probe point writer class name.
-# BENCHMARK_WRITER=
-
-# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
-SERVER_HOSTS=localhost,localhost
-
-# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
-DRIVER_HOSTS=localhost
-
-# Remote username.
-# REMOTE_USER=
-
-# Number of nodes, used to wait for the specified number of nodes to start.
-nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
-
-# Run configuration.
-# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
-CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutIndex1Benchmark -sn IgniteNode -ds atomic-indexed1-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutIndex2Benchmark -sn IgniteNode -ds atomic-indexed2-put,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 1 -w 60 -d 300 -t 64 -sm PRIMARY_SYNC -dn IgnitePutIndex8Benchmark -sn IgniteNode -ds atomic-indexed8-put\
-"

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.java
deleted file mode 100644
index 65dd39a..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex1Benchmark.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.*;
-
-/**
- * Ignite benchmark that performs put operations for entity with indexed fields.
- */
-public class IgnitePutIndex1Benchmark extends IgniteCacheAbstractBenchmark {
-    /** {@inheritDoc} */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        int key = nextRandom(args.range());
-
-        cache.put(key, new Person1(key));
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteCache<Integer, Object> cache() {
-        return ignite().cache("atomic-index");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.java
deleted file mode 100644
index b4cf86f..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndex2Benchmark.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.*;
-
-/**
- * Ignite benchmark that performs put operations for entity with indexed fields.
- */
-public class IgnitePutIndex2Benchmark extends IgniteCacheAbstractBenchmark {
-    /** {@inheritDoc} */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        int key = nextRandom(args.range());
-
-        cache.put(key, new Person2(key, key + 1));
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteCache<Integer, Object> cache() {
-        return ignite().cache("atomic-index");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue1Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue1Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue1Benchmark.java
new file mode 100644
index 0000000..68de785
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue1Benchmark.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.*;
+
+/**
+ * Ignite benchmark that performs put operations for entity with indexed fields.
+ */
+public class IgnitePutIndexedValue1Benchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new Person1(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue2Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue2Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue2Benchmark.java
new file mode 100644
index 0000000..3256ee3
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue2Benchmark.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.*;
+
+/**
+ * Ignite benchmark that performs put operations for entity with indexed fields.
+ */
+public class IgnitePutIndexedValue2Benchmark extends IgniteCacheAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        int key = nextRandom(args.range());
+
+        cache.put(key, new Person2(key));
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("atomic-index");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java
index b51ff34..42aa67d 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutIndexedValue8Benchmark.java
@@ -30,7 +30,7 @@ public class IgnitePutIndexedValue8Benchmark extends IgniteCacheAbstractBenchmar
     @Override public boolean test(Map<Object, Object> ctx) throws Exception {
         int key = nextRandom(args.range());
 
-        cache.put(key, new Person8(key, key + 1, key + 2, key + 3, key + 4, key + 5, key + 6, key + 7));
+        cache.put(key, new Person8(key));
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
index 0b2a0bf..91d1ccc 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
@@ -30,35 +30,12 @@ public class Person1 implements Serializable {
     private int val1;
 
     /**
-     * Empty constructor.
-     */
-    public Person1() {
-        // No-op.
-    }
-
-    /**
      * Constructs.
      *
-     * @param val1 Indexed value.
-     */
-    public Person1(int val1) {
-        this.val1 = val1;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal1() {
-        return val1;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val1 Value.
+     * @param val Indexed value.
      */
-    public void setVal1(int val1) {
-        this.val1 = val1;
+    public Person1(int val) {
+        this.val1 = val;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
index 3b1cb9e..dc5729c 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
@@ -34,50 +34,13 @@ public class Person2 implements Serializable {
     private int val2;
 
     /**
-     * Empty constructor.
-     */
-    public Person2() {
-        // No-op.
-    }
-
-    /**
      * Constructs.
-     */
-    public Person2(int val1, int val2) {
-        this.val1 = val1;
-        this.val2 = val2;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal1() {
-        return val1;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val1 Value.
-     */
-    public void setVal1(int val1) {
-        this.val1 = val1;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal2() {
-        return val2;
-    }
-
-    /**
-     * Set value.
      *
-     * @param val2 Value.
+     * @param val Value.
      */
-    public void setVal2(int val2) {
-        this.val2 = val2;
+    public Person2(int val) {
+        this.val1 = val;
+        this.val2 = val + 1;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1a6948e8/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
index b8e3f7c..92d65ac 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
@@ -58,152 +58,19 @@ public class Person8 implements Serializable {
     private int val8;
 
     /**
-     * Empty constructor.
-     */
-    public Person8() {
-        // No-op.
-    }
-
-    /**
      * Constructs.
-     */
-    public Person8(int val1, int val2, int val3, int val4, int val5, int val6, int val7, int val8) {
-        this.val1 = val1;
-        this.val2 = val2;
-        this.val3 = val3;
-        this.val4 = val4;
-        this.val5 = val5;
-        this.val6 = val6;
-        this.val7 = val7;
-        this.val8 = val8;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal1() {
-        return val1;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val1 Value.
-     */
-    public void setVal1(int val1) {
-        this.val1 = val1;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal2() {
-        return val2;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val2 Value.
-     */
-    public void setVal2(int val2) {
-        this.val2 = val2;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal3() {
-        return val3;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val3 Value.
-     */
-    public void setVal3(int val3) {
-        this.val3 = val3;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal4() {
-        return val4;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val4 Value.
-     */
-    public void setVal4(int val4) {
-        this.val4 = val4;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal5() {
-        return val5;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val5 Value.
-     */
-    public void setVal5(int val5) {
-        this.val5 = val5;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal6() {
-        return val6;
-    }
-
-    /**
-     * Set value.
      *
-     * @param val6 Value.
-     */
-    public void setVal6(int val6) {
-        this.val6 = val6;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal7() {
-        return val7;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val7 Value.
-     */
-    public void setVal7(int val7) {
-        this.val7 = val7;
-    }
-
-    /**
-     * @return Get value.
-     */
-    public int getVal8() {
-        return val8;
-    }
-
-    /**
-     * Set value.
-     *
-     * @param val8 Value.
-     */
-    public void setVal8(int val8) {
-        this.val8 = val8;
+     * @param val Indexed value.
+     */
+    public Person8(int val) {
+        this.val1 = val;
+        this.val2 = val + 1;
+        this.val3 = val + 2;
+        this.val4 = val + 3;
+        this.val5 = val + 4;
+        this.val6 = val + 5;
+        this.val7 = val + 6;
+        this.val8 = val + 7;
     }
 
     /** {@inheritDoc} */