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

[01/50] incubator-ignite git commit: #ignite-237: add node attributes.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-187 4ba28629f -> ba3fcbe1f


#ignite-237: add node attributes.


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

Branch: refs/heads/ignite-187
Commit: 148ca09572b336cfe4c55683e059ca13fa8cd238
Parents: d254da4
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 15:53:51 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 15:53:51 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheAttributes.java       | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/148ca095/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 571ae25..8ecd84c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -191,6 +192,13 @@ public class GridCacheAttributes implements Externalizable {
         CacheAffinityFunction aff = cfg.getAffinity();
 
         if (aff != null) {
+            if (aff instanceof CacheRendezvousAffinityFunction) {
+                CacheRendezvousAffinityFunction aff0 = (CacheRendezvousAffinityFunction) aff;
+
+                affInclNeighbors = aff0.isExcludeNeighbors();
+                affHashIdRslvrClsName = className(aff0.getHashIdResolver());
+            }
+
             affPartsCnt = aff.partitions();
             affClsName = className(aff);
         }


[29/50] incubator-ignite git commit: #ignite-239: remove replicaCount from GridClientNode and GridClientNodeBean.

Posted by ak...@apache.org.
#ignite-239: remove replicaCount from GridClientNode and GridClientNodeBean.


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

Branch: refs/heads/ignite-187
Commit: 5f620d62191188ff4dcbe55c5d0399603e494b45
Parents: 6c67bdf
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 12:40:13 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 12:40:13 2015 +0300

----------------------------------------------------------------------
 .../impl/ClientPartitionAffinitySelfTest.java   |  4 +---
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |  4 ----
 .../ignite/internal/client/GridClientNode.java  |  8 -------
 .../client/GridClientPartitionAffinity.java     |  2 +-
 .../client/impl/GridClientNodeImpl.java         | 22 -------------------
 .../connection/GridClientNioTcpConnection.java  |  3 +--
 .../rest/client/message/GridClientNodeBean.java | 23 --------------------
 7 files changed, 3 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
index 7ece534..6af1c4c 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
@@ -152,7 +152,7 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
 
         aff.setHashIdResolver(new GridClientPartitionAffinity.HashIdResolver() {
             @Override public Object getHashId(GridClientNode node) {
-                return node.replicaCount();
+                return 1;
             }
         });
 
@@ -258,7 +258,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
     private GridClientNode createNode(String nodeId, int replicaCnt) {
         return GridClientNodeImpl.builder()
             .nodeId(UUID.fromString(nodeId))
-            .replicaCount(replicaCnt)
             .build();
     }
 
@@ -276,7 +275,6 @@ public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
 
             nodes.add(GridClientNodeImpl.builder()
                 .nodeId(nodeId)
-                .replicaCount(replicaCnt)
                 .build());
 
             ClusterNode srvNode = new TestRichNode(nodeId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
index bb3710e..fda7e99 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientTcpUnreachableMultiNodeSelfTest.java
@@ -85,10 +85,6 @@ public class ClientTcpUnreachableMultiNodeSelfTest extends ClientTcpMultiNodeSel
                     return node.caches();
                 }
 
-                @Override public int replicaCount() {
-                    return node.replicaCount();
-                }
-
                 @Override public List<String> tcpAddresses() {
                     return node.tcpAddresses();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
index ac2fe86..74e612d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientNode.java
@@ -99,14 +99,6 @@ public interface GridClientNode {
     public Map<String, GridClientCacheMode> caches();
 
     /**
-     * Gets node replica count for consistent hash ring (valid only for
-     * {@code PARTITIONED} caches).
-     *
-     * @return Node replica count for consistent hash ring.
-     */
-    public int replicaCount();
-
-    /**
      * Gets collection of addresses on which REST binary protocol is bound.
      *
      * @param proto Protocol for which addresses are obtained.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
index a0fad61..b747bc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientPartitionAffinity.java
@@ -274,7 +274,7 @@ public class GridClientPartitionAffinity implements GridClientDataAffinity, Grid
         nodeInfo = new NodeInfo(nodeId, hashIdRslvr == null ? nodeId : hashIdRslvr.getHashId(n));
 
         addedNodes.put(nodeId, nodeInfo);
-        nodeHash.addNode(nodeInfo, n.replicaCount());
+        nodeHash.addNode(nodeInfo, 1);
 
         return nodeInfo;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
index 6692550..9c59b84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/GridClientNodeImpl.java
@@ -53,9 +53,6 @@ public class GridClientNodeImpl implements GridClientNode {
     /** Node caches. */
     private Map<String, GridClientCacheMode> caches = Collections.emptyMap();
 
-    /** Replica count for partitioned cache. */
-    private int replicaCnt;
-
     /** Connectable property. */
     private boolean connectable;
 
@@ -95,7 +92,6 @@ public class GridClientNodeImpl implements GridClientNode {
             .tcpAddresses(from.tcpAddresses())
             .tcpPort(from.tcpPort())
             .caches(from.caches())
-            .replicaCount(from.replicaCount())
             .connectable(from.connectable());
 
         if (!skipAttrs)
@@ -154,11 +150,6 @@ public class GridClientNodeImpl implements GridClientNode {
     }
 
     /** {@inheritDoc} */
-    @Override public int replicaCount() {
-        return replicaCnt;
-    }
-
-    /** {@inheritDoc} */
     @Override public Collection<InetSocketAddress> availableAddresses(GridClientProtocol proto,
         boolean filterResolved) {
         Collection<String> addrs;
@@ -368,19 +359,6 @@ public class GridClientNodeImpl implements GridClientNode {
             return this;
         }
 
-
-        /**
-         * Sets replica count for node on consistent hash ring.
-         *
-         * @param replicaCnt Replica count.
-         * @return This for chaining.
-         */
-        public Builder replicaCount(int replicaCnt) {
-            impl.replicaCnt = replicaCnt;
-
-            return this;
-        }
-
         /**
          * Sets connectable property.
          *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index d84bca5..44d10f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -868,8 +868,7 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             .nodeId(nodeBean.getNodeId())
             .consistentId(nodeBean.getConsistentId())
             .tcpAddresses(nodeBean.getTcpAddresses())
-            .tcpPort(nodeBean.getTcpPort())
-            .replicaCount(nodeBean.getReplicaCount());
+            .tcpPort(nodeBean.getTcpPort());
 
         Map<String, GridClientCacheMode> caches = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f620d62/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
index 6605a15..833abd9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientNodeBean.java
@@ -56,9 +56,6 @@ public class GridClientNodeBean implements Externalizable {
     /** Node caches. */
     private Map<String, String> caches;
 
-    /** Default replica count for partitioned cache. */
-    private int replicaCnt;
-
     /**
      * Gets node ID.
      *
@@ -209,24 +206,6 @@ public class GridClientNodeBean implements Externalizable {
     }
 
     /**
-     * Gets node replica count on consistent hash ring.
-     *
-     * @return Node replica count.
-     */
-    public int getReplicaCount() {
-        return replicaCnt;
-    }
-
-    /**
-     * Sets node replica count on consistent hash ring.
-     *
-     * @param replicaCnt Node replica count.
-     */
-    public void setReplicaCount(int replicaCnt) {
-        this.replicaCnt = replicaCnt;
-    }
-
-    /**
      * Sets REST binary protocol port.
      *
      * @param tcpPort Port on which REST binary protocol is bound.
@@ -257,7 +236,6 @@ public class GridClientNodeBean implements Externalizable {
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(tcpPort);
         out.writeInt(0); // Jetty port.
-        out.writeInt(replicaCnt);
 
         U.writeString(out, dfltCacheMode);
 
@@ -279,7 +257,6 @@ public class GridClientNodeBean implements Externalizable {
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         tcpPort = in.readInt();
         in.readInt(); // Jetty port.
-        replicaCnt = in.readInt();
 
         dfltCacheMode = U.readString(in);
 


[25/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239


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

Branch: refs/heads/ignite-187
Commit: 0b1eccf96a260b3208c153ead6b4e314798726ad
Parents: 0dfe349 c9f46c1
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 11:40:06 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 11:40:06 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |  90 ++++++++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 137 +++++++++++--------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 116 ++++++++++++++++
 3 files changed, 275 insertions(+), 68 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-ignite git commit: #ignite-237: remove unused imports.

Posted by ak...@apache.org.
#ignite-237: remove unused imports.


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

Branch: refs/heads/ignite-187
Commit: 8e587386d284373dbd6ac310dfcb2a9369236867
Parents: 5692d7e
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 19:46:58 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 19:46:58 2015 +0300

----------------------------------------------------------------------
 .../internal/client/impl/ClientPartitionAffinitySelfTest.java    | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e587386/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
index dde345e..7ece534 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
@@ -17,12 +17,8 @@
 
 package org.apache.ignite.internal.client.impl;
 
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.client.*;
-import org.apache.ignite.internal.processors.affinity.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 


[24/50] incubator-ignite git commit: #ignite-237: remove debug.

Posted by ak...@apache.org.
#ignite-237: remove debug.


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

Branch: refs/heads/ignite-187
Commit: 6deba53c2747432c36503ba83932199a943cdcb7
Parents: 6531cbd
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 22:59:47 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 22:59:47 2015 +0300

----------------------------------------------------------------------
 ...GridCacheContinuousQueryAbstractSelfTest.java | 19 +++----------------
 1 file changed, 3 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6deba53c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 850a213..db1ac5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -159,23 +159,10 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             }
         }
 
-        boolean allEmpty = true;
-
-        for (int i = 0; i < gridCount(); i++) {
-            Set<Cache.Entry<Object, Object>> entries = ((IgniteKernal) grid(i)).cache(null).entrySet();
-
-            for (Cache.Entry entry : entries) {
-                boolean primary = grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey());
-
-                boolean backup = grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey());
-
-                info("Not removed entry [grid=" + i + ", primary=" + primary + ", backup=" + backup + ']');
-
-                allEmpty = false;
-            }
-        }
+        for (int i = 0; i < gridCount(); i++)
+            assertEquals("Cache is not empty: " + ((IgniteKernal)grid(i)).cache(null).entrySet(), 0,
+                ((IgniteKernal)grid(i)).cache(null).size());
 
-        assertTrue(allEmpty);
 
         for (int i = 0; i < gridCount(); i++) {
             GridContinuousProcessor proc = ((IgniteKernal)grid(i)).context().continuous();


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

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

Conflicts:
	modules/core/src/main/java/org/apache/ignite/marshaller/optimized/optimized-classnames.properties


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

Branch: refs/heads/ignite-187
Commit: 0d70de861e9f9beb8bc237ce81909efccd18866c
Parents: e50d1ea f053746
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:28:17 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:28:17 2015 +0300

----------------------------------------------------------------------
 config/ignite-log4j.xml                         |    2 +-
 .../datagrid/CacheContinuousQueryExample.java   |    2 +-
 ipc/shmem/Makefile.am                           |   15 +
 ipc/shmem/igniteshmem/Makefile.am               |   15 +
 ipc/shmem/include/Makefile.am                   |   15 +
 modules/clients/src/test/keystore/generate.sh   |   15 +-
 .../ignite/cache/query/ContinuousQuery.java     |   18 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |    3 +
 .../processors/cache/IgniteCacheProxy.java      |    6 +-
 .../optimized-classnames.previous.properties    |   15 +
 .../optimized/optimized-classnames.properties   | 1565 +-----------------
 .../TcpDiscoveryCustomEventMessage.java         |    3 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 modules/extdata/p2p/pom.xml                     |    6 -
 modules/hibernate/pom.xml                       |    6 -
 .../HibernateReadWriteAccessStrategy.java       |   81 +-
 modules/indexing/pom.xml                        |    6 -
 modules/jta/pom.xml                             |    6 -
 modules/scalar/pom.xml                          |    6 -
 modules/spring/pom.xml                          |    6 -
 modules/visor-console/pom.xml                   |    7 -
 modules/web/pom.xml                             |    6 -
 modules/winservice/IgniteService.sln            |    2 +-
 .../IgniteService/IgniteService.csproj          |    2 +-
 .../config/benchmark-atomic-win.properties      |   15 +
 .../config/benchmark-atomic.properties          |   15 +
 .../config/benchmark-compute-win.properties     |   15 +
 .../config/benchmark-compute.properties         |   15 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-query-win.properties       |   15 +
 .../yardstick/config/benchmark-query.properties |   15 +
 .../config/benchmark-tx-win.properties          |   15 +
 .../yardstick/config/benchmark-tx.properties    |   15 +
 .../yardstick/config/benchmark-win.properties   |   15 +
 modules/yardstick/config/benchmark.properties   |   15 +
 pom.xml                                         |  150 +-
 37 files changed, 421 insertions(+), 1712 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d70de86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------


[03/50] incubator-ignite git commit: #ignite-239: small fixes.

Posted by ak...@apache.org.
#ignite-239: small fixes.


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

Branch: refs/heads/ignite-187
Commit: 4f0feeecb8c4c48ee7e7ffe4679ff0ac28ef3a0b
Parents: 5830deb
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 16:15:17 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 16:15:17 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 72 ++++++++++----------
 1 file changed, 37 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f0feeec/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 32a3d71..f39cd7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -119,9 +119,6 @@ public class IgnitionEx {
     /** */
     private static volatile boolean daemon;
 
-    /** */
-    private static final String[] EMPTY_STR_ARR = new String[0];
-
     /**
      * Checks runtime version to be 1.7.x or 1.8.x.
      * This will load pretty much first so we must do these checks here.
@@ -1138,6 +1135,9 @@ public class IgnitionEx {
         private static final Map<MBeanServer, GridMBeanServerData> mbeans =
             new HashMap<>();
 
+        /** */
+        private static final String[] EMPTY_STR_ARR = new String[0];
+
         /** Grid name. */
         private final String name;
 
@@ -1297,6 +1297,17 @@ public class IgnitionEx {
 
             IgniteConfiguration cfg = startCtx.config() != null ? startCtx.config() : new IgniteConfiguration();
 
+            String ggHome = cfg.getIgniteHome();
+
+            // Set Ignite home.
+            if (ggHome == null)
+                ggHome = U.getIgniteHome();
+            else
+                // If user provided IGNITE_HOME - set it as a system property.
+                U.setIgniteHome(ggHome);
+
+            U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
+
             // Ensure invariant.
             // It's a bit dirty - but this is a result of late refactoring
             // and I don't want to reshuffle a lot of code.
@@ -1317,20 +1328,18 @@ public class IgnitionEx {
 
             log = cfgLog.getLogger(G.class);
 
-            IgniteConfiguration myCfg = initializeDefaultConfiguration(cfg);
-
-            myCfg.setGridLogger(cfgLog);
+            // Check Ignite home folder (after log is available).
+            if (ggHome != null) {
+                File ggHomeFile = new File(ggHome);
 
-            if (myCfg.getConnectorConfiguration() != null) {
-                restExecSvc = new IgniteThreadPoolExecutor(
-                    "rest-" + cfg.getGridName(),
-                    myCfg.getConnectorConfiguration().getThreadPoolSize(),
-                    myCfg.getConnectorConfiguration().getThreadPoolSize(),
-                    ConnectorConfiguration.DFLT_KEEP_ALIVE_TIME,
-                    new LinkedBlockingQueue<Runnable>(ConnectorConfiguration.DFLT_THREADPOOL_QUEUE_CAP)
-                );
+                if (!ggHomeFile.exists() || !ggHomeFile.isDirectory())
+                    throw new IgniteCheckedException("Invalid Ignite installation home folder: " + ggHome);
             }
 
+            IgniteConfiguration myCfg = initializeConfiguration(cfg);
+
+            myCfg.setGridLogger(cfgLog);
+
             // Validate segmentation configuration.
             GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
 
@@ -1417,6 +1426,16 @@ public class IgnitionEx {
                 0,
                 new LinkedBlockingQueue<Runnable>());
 
+            if (myCfg.getConnectorConfiguration() != null) {
+                restExecSvc = new IgniteThreadPoolExecutor(
+                    "rest-" + myCfg.getGridName(),
+                    myCfg.getConnectorConfiguration().getThreadPoolSize(),
+                    myCfg.getConnectorConfiguration().getThreadPoolSize(),
+                    ConnectorConfiguration.DFLT_KEEP_ALIVE_TIME,
+                    new LinkedBlockingQueue<Runnable>(ConnectorConfiguration.DFLT_THREADPOOL_QUEUE_CAP)
+                );
+            }
+
             utilityCacheExecSvc = new IgniteThreadPoolExecutor(
                 "utility-" + cfg.getGridName(),
                 DFLT_SYSTEM_CORE_THREAD_CNT,
@@ -1499,10 +1518,12 @@ public class IgnitionEx {
          * @param cfg Ignite configuration copy to.
          * @return New ignite configuration.
          */
-        private IgniteConfiguration initializeDefaultConfiguration(IgniteConfiguration cfg)
+        private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg)
             throws IgniteCheckedException {
             IgniteConfiguration myCfg = new IgniteConfiguration(cfg);
+
             initializeDefaultConfigurationParameters(myCfg);
+
             return myCfg;
         }
 
@@ -1510,26 +1531,7 @@ public class IgnitionEx {
          * Initialize default parameters.
          */
         public void initializeDefaultConfigurationParameters(IgniteConfiguration cfg) throws IgniteCheckedException {
-            // Set Ignite home.
-            String ggHome = cfg.getIgniteHome();
-
-            if (ggHome == null)
-                ggHome = U.getIgniteHome();
-            else
-                // If user provided IGNITE_HOME - set it as a system property.
-                U.setIgniteHome(ggHome);
-
-            U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
-
-            // Check Ignite home folder (after log is available).
-            if (ggHome != null) {
-                File ggHomeFile = new File(ggHome);
-
-                if (!ggHomeFile.exists() || !ggHomeFile.isDirectory())
-                    throw new IgniteCheckedException("Invalid Ignite installation home folder: " + ggHome);
-            }
-
-            cfg.setIgniteHome(ggHome);
+            cfg.setIgniteHome(U.getIgniteHome());
 
             // Local host.
             String locHost = IgniteSystemProperties.getString(IGNITE_LOCAL_HOST);


[28/50] incubator-ignite git commit: # ignite-382 Changed version.

Posted by ak...@apache.org.
# ignite-382 Changed version.


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

Branch: refs/heads/ignite-187
Commit: f252ee4177c3f1cf4b8eb47014e930da696600c2
Parents: 13e2d1f
Author: anovikov <an...@gridgain.com>
Authored: Mon Mar 2 16:36:11 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Mon Mar 2 16:36:11 2015 +0700

----------------------------------------------------------------------
 examples/pom-standalone.xml                       | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 pom.xml                                           | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f252ee41/examples/pom-standalone.xml
----------------------------------------------------------------------
diff --git a/examples/pom-standalone.xml b/examples/pom-standalone.xml
index 47df6bd..5de7433 100644
--- a/examples/pom-standalone.xml
+++ b/examples/pom-standalone.xml
@@ -28,7 +28,7 @@
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-        <ignite.version>1.0.1-RC1-SNAPSHOT</ignite.version>
+        <ignite.version>1.0.0-RC2-SNAPSHOT</ignite.version>
     </properties>
 
     <groupId>org.apache.ignite</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f252ee41/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index caa7553..4295dce 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.0.1-rc1
+ignite.version=1.0.0-rc2
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f252ee41/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98f649d..b0ebc44 100644
--- a/pom.xml
+++ b/pom.xml
@@ -33,7 +33,7 @@
     </parent>
 
     <properties>
-        <ignite.version>1.0.1-RC1-SNAPSHOT</ignite.version>
+        <ignite.version>1.0.0-RC2-SNAPSHOT</ignite.version>
         <ignite.edition>fabric</ignite.edition>
         <hadoop.version>2.4.1</hadoop.version>
         <spring.version>4.1.0.RELEASE</spring.version>
@@ -574,7 +574,7 @@
                 </plugins>
             </build>
         </profile>
-        
+
         <profile>
             <id>test</id>
             <build>


[11/50] incubator-ignite git commit: #ignite-239: review.

Posted by ak...@apache.org.
#ignite-239: 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/8476463b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8476463b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8476463b

Branch: refs/heads/ignite-187
Commit: 8476463b9145eff58d28da7e57d83d96e50a066c
Parents: 4ff4fd1
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:17:07 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:17:07 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgnitionEx.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8476463b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 6b8b197..c7e7cb6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1306,8 +1306,6 @@ public class IgnitionEx {
             // and I don't want to reshuffle a lot of code.
             assert F.eq(name, cfg.getGridName());
 
-            IgniteConfiguration myCfg = initializeConfiguration(cfg);
-
             // Validate segmentation configuration.
             GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
 
@@ -1318,6 +1316,8 @@ public class IgnitionEx {
                     "on start?) [segPlc=" + segPlc + ", wait=false]");
             }
 
+            IgniteConfiguration myCfg = initializeConfiguration(cfg);
+
             // Ensure that SPIs support multiple grid instances, if required.
             if (!startCtx.single()) {
                 ensureMultiInstanceSupport(myCfg.getDeploymentSpi());


[36/50] incubator-ignite git commit: Merge branch 'ignite-270' into sprint-2

Posted by ak...@apache.org.
Merge branch 'ignite-270' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: 2511ebe18c564e43e642bc879ffe44b02497c90a
Parents: b0e6ab1 18548b2
Author: anovikov <an...@gridgain.com>
Authored: Mon Mar 2 21:43:59 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Mon Mar 2 21:43:59 2015 +0700

----------------------------------------------------------------------
 modules/scalar/pom.xml | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2511ebe1/modules/scalar/pom.xml
----------------------------------------------------------------------


[35/50] incubator-ignite git commit: ignite-352 review

Posted by ak...@apache.org.
ignite-352 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/b0e6ab17
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/b0e6ab17
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/b0e6ab17

Branch: refs/heads/ignite-187
Commit: b0e6ab17c284850190012e7c120a95c34316b982
Parents: 403d60f
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Mar 2 17:28:39 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Mar 2 17:28:39 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheMetricsImpl.java      |  4 ++
 .../dht/atomic/GridDhtAtomicCache.java          |  2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  2 +-
 .../cache/transactions/IgniteTxManager.java     |  7 +++-
 .../cache/GridCacheAbstractMetricsSelfTest.java | 42 ++++++++++++++++++++
 5 files changed, 53 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0e6ab17/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 0de039b..446070e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -460,6 +460,8 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /**
      * Transaction commit callback.
+     *
+     * @param duration the time taken in nanoseconds.
      */
     public void onTxCommit(long duration) {
         txCommits.incrementAndGet();
@@ -472,6 +474,8 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /**
      * Transaction rollback callback.
+     *
+     * @param duration the time taken in nanoseconds.
      */
     public void onTxRollback(long duration) {
         txRollbacks.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0e6ab17/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 9f9af31..add51bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -990,7 +990,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                 if (!success)
                     break;
-                else if (!skipVals)
+                else if (!skipVals && ctx.config().isStatisticsEnabled())
                     metrics0().onRead(true);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0e6ab17/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index a59b6aa..cdb1759 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -318,7 +318,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
 
                 if (!success)
                     break;
-                else if (!skipVals)
+                else if (!skipVals && ctx.config().isStatisticsEnabled())
                     ctx.cache().metrics0().onRead(true);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0e6ab17/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index cf32dcc..af57ce4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -1243,7 +1243,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
                     GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
 
                     if (cacheCtx.cache().configuration().isStatisticsEnabled())
-                        cacheCtx.cache().metrics0().onTxCommit(System.nanoTime() - tx.startTime());
+                        // Convert start time from ms to ns.
+                        cacheCtx.cache().metrics0().onTxCommit((U.currentTimeMillis() - tx.startTime()) * 1000);
                 }
             }
 
@@ -1316,7 +1317,9 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
                 for (int cacheId : tx.activeCacheIds()) {
                     GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
 
-                    cacheCtx.cache().metrics0().onTxRollback(System.nanoTime() - tx.startTime());
+                    if (cacheCtx.cache().configuration().isStatisticsEnabled())
+                        // Convert start time from ms to ns.
+                        cacheCtx.cache().metrics0().onTxRollback((U.currentTimeMillis() - tx.startTime()) * 1000);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0e6ab17/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
index d1c5e12..a7fb34a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
@@ -103,6 +103,48 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testGetMetricsDisable() throws Exception {
+        // Disable statistics.
+        for (int i = 0; i < gridCount(); i++) {
+            Ignite g = grid(i);
+
+            g.jcache(null).getConfiguration(CacheConfiguration.class).setStatisticsEnabled(false);
+        }
+
+        IgniteCache<Object, Object> jcache = grid(0).jcache(null);
+
+        // Write to cache.
+        for (int i = 0; i < KEY_CNT; i++)
+            jcache.put(i, i);
+
+        // Get from cache.
+        for (int i = 0; i < KEY_CNT; i++)
+            jcache.get(i);
+
+        // Remove from cache.
+        for (int i = 0; i < KEY_CNT; i++)
+            jcache.remove(i);
+
+        // Assert that statistics is clear.
+        for (int i = 0; i < gridCount(); i++) {
+            CacheMetrics m = grid(i).jcache(null).metrics();
+
+            assertEquals(m.getCacheGets(), 0);
+            assertEquals(m.getCachePuts(), 0);
+            assertEquals(m.getCacheRemovals(), 0);
+            assertEquals(m.getCacheHits(), 0);
+            assertEquals(m.getCacheMisses(), 0);
+            assertEquals(m.getAverageGetTime(), 0f);
+            assertEquals(m.getAverageRemoveTime(), 0f);
+            assertEquals(m.getAveragePutTime(), 0f);
+            assertEquals(m.getAverageTxCommitTime(), 0f);
+            assertEquals(m.getAverageTxRollbackTime(), 0f);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetMetricsSnapshot() throws Exception {
         IgniteCache<Object, Object> cache = grid(0).jcache(null);
 


[13/50] incubator-ignite git commit: #ignite-237: remove old GridCacheAttributes.

Posted by ak...@apache.org.
#ignite-237: remove old GridCacheAttributes.


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

Branch: refs/heads/ignite-187
Commit: 936b8792b9bd6839c7620f8f1021dcd82952aef4
Parents: 0d70de8
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:31:48 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:31:48 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAttributes.java   | 24 --------------------
 .../processors/cache/GridCacheProcessor.java    |  8 -------
 2 files changed, 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/936b8792/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 8ecd84c..94ae1fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -119,12 +119,6 @@ public class GridCacheAttributes implements Externalizable {
     private int affKeyBackups = -1;
 
     /** */
-    private int affReplicas = -1;
-
-    /** */
-    private String affReplicaCntAttrName;
-
-    /** */
     private String affHashIdRslvrClsName;
 
     /** */
@@ -294,13 +288,6 @@ public class GridCacheAttributes implements Externalizable {
     }
 
     /**
-     * @return Affinity replicas.
-     */
-    public int affinityReplicas() {
-        return affReplicas;
-    }
-
-    /**
      * @return Affinity partitions count.
      */
     public int affinityPartitionsCount() {
@@ -308,13 +295,6 @@ public class GridCacheAttributes implements Externalizable {
     }
 
     /**
-     * @return Aff replicas count attr name.
-     */
-    public String affinityReplicaCountAttrName() {
-        return affReplicaCntAttrName;
-    }
-
-    /**
      * @return Affinity hash ID resolver class name.
      */
     public String affinityHashIdResolverClassName() {
@@ -537,8 +517,6 @@ public class GridCacheAttributes implements Externalizable {
         out.writeBoolean(affInclNeighbors);
         out.writeInt(affKeyBackups);
         out.writeInt(affPartsCnt);
-        out.writeInt(affReplicas);
-        U.writeString(out, affReplicaCntAttrName);
         U.writeString(out, affHashIdRslvrClsName);
 
         U.writeString(out, evictFilterClsName);
@@ -581,8 +559,6 @@ public class GridCacheAttributes implements Externalizable {
         affInclNeighbors = in.readBoolean();
         affKeyBackups = in.readInt();
         affPartsCnt = in.readInt();
-        affReplicas = in.readInt();
-        affReplicaCntAttrName = U.readString(in);
         affHashIdRslvrClsName = U.readString(in);
 
         evictFilterClsName = U.readString(in);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/936b8792/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index f00a853..3e5e124 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1186,14 +1186,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                                 "Affinity key backups", locAttr.affinityKeyBackups(),
                                 rmtAttr.affinityKeyBackups(), true);
 
-                            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityReplicas",
-                                "Affinity replicas", locAttr.affinityReplicas(),
-                                rmtAttr.affinityReplicas(), true);
-
-                            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityReplicaCountAttrName",
-                                "Affinity replica count attribute name", locAttr.affinityReplicaCountAttrName(),
-                                rmtAttr.affinityReplicaCountAttrName(), true);
-
                             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheAffinity.hashIdResolver",
                                 "Partitioned cache affinity hash ID resolver class",
                                 locAttr.affinityHashIdResolverClassName(), rmtAttr.affinityHashIdResolverClassName(),


[42/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.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/9efcd803
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9efcd803
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9efcd803

Branch: refs/heads/ignite-187
Commit: 9efcd8035dd74977f44ce7ab6ebd989b57e57482
Parents: 5a76d70 b2675bc
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 18:03:07 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 18:03:07 2015 +0300

----------------------------------------------------------------------
 examples/pom-standalone.xml                     |   2 +-
 .../java/org/apache/ignite/IgniteCluster.java   |   8 +-
 .../configuration/IgniteConfiguration.java      |   5 +-
 .../ignite/internal/GridKernalContext.java      |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |  14 +-
 .../apache/ignite/internal/IgniteKernal.java    |  23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++++----------
 .../affinity/GridAffinityAssignmentCache.java   |  13 +-
 .../processors/cache/CacheMetricsImpl.java      |   4 +
 .../processors/cache/GridCacheContext.java      |  19 +
 .../processors/cache/GridCacheMapEntry.java     |   9 +
 .../processors/cache/GridCacheProcessor.java    |  15 +
 .../processors/cache/GridCacheStoreManager.java |  12 +-
 .../GridDistributedCacheAdapter.java            |   2 +
 .../distributed/dht/GridDhtCacheEntry.java      |   3 +
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../processors/cluster/ClusterProcessor.java    |  46 ++
 .../dataload/IgniteDataLoaderImpl.java          |  29 +-
 .../ignite/internal/visor/cache/VisorCache.java | 190 ++----
 .../cache/VisorCacheAffinityConfiguration.java  |  51 +-
 .../visor/cache/VisorCacheConfiguration.java    | 484 +++------------
 .../cache/VisorCacheDefaultConfiguration.java   |  27 +-
 .../cache/VisorCacheEvictionConfiguration.java  |  81 +--
 .../cache/VisorCacheNearConfiguration.java      |  42 +-
 .../cache/VisorCachePreloadConfiguration.java   |  54 +-
 .../cache/VisorCacheStoreConfiguration.java     | 148 ++++-
 .../VisorCacheWriteBehindConfiguration.java     | 137 -----
 .../visor/node/VisorAtomicConfiguration.java    |  27 +-
 .../visor/node/VisorBasicConfiguration.java     | 180 +-----
 .../node/VisorCacheQueryConfiguration.java      |  45 +-
 .../node/VisorExecutorServiceConfiguration.java |  54 +-
 .../visor/node/VisorGridConfiguration.java      | 177 +-----
 .../visor/node/VisorIgfsConfiguration.java      | 244 +-------
 .../visor/node/VisorLifecycleConfiguration.java |   9 +-
 .../visor/node/VisorMetricsConfiguration.java   |  29 +-
 .../node/VisorPeerToPeerConfiguration.java      |  28 +-
 .../visor/node/VisorQueryConfiguration.java     |  65 +-
 .../visor/node/VisorRestConfiguration.java      |  80 +--
 .../node/VisorSegmentationConfiguration.java    |  45 +-
 .../visor/node/VisorSpisConfiguration.java      |  92 +--
 .../node/VisorTransactionConfiguration.java     |  62 +-
 .../internal/visor/util/VisorTaskUtils.java     |   4 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/store/jdbc/Ignite.xml  |  63 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    | 440 ++++++++++++++
 .../cache/GridCacheAbstractMetricsSelfTest.java |  42 ++
 ...acheAbstractUsersAffinityMapperSelfTest.java | 207 +++++++
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCachePartitionedLocalStoreSelfTest.java |  51 ++
 ...chePartitionedOffHeapLocalStoreSelfTest.java |  56 ++
 .../GridCacheReplicatedLocalStoreSelfTest.java  |  51 ++
 ...heReplicatedUsersAffinityMapperSelfTest.java |  45 ++
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |  51 ++
 .../GridCacheTxUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  38 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 modules/scalar/pom.xml                          |   6 +
 .../ignite/schema/generator/XmlGenerator.java   |   8 +-
 .../apache/ignite/schema/model/PojoField.java   |  11 +-
 .../apache/ignite/schema/load/model/Ignite.xml  | 133 +++--
 .../commands/cache/VisorCacheCommand.scala      |  25 +-
 .../yardstick/config/ignite-store-config.xml    |  15 +-
 pom.xml                                         |   4 +-
 67 files changed, 2105 insertions(+), 2415 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9efcd803/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9efcd803/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 47a2cf4,0f6a84f..6ff2d2d
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@@ -69,10 -71,12 +69,11 @@@ public class VisorCacheAffinityConfigur
  
          VisorCacheAffinityConfiguration cfg = new VisorCacheAffinityConfiguration();
  
-         cfg.function(compactClass(aff));
-         cfg.mapper(compactClass(ccfg.getAffinityMapper()));
-         cfg.partitionedBackups(ccfg.getBackups());
-         cfg.excludeNeighbors(excludeNeighbors);
+         cfg.function = compactClass(aff);
+         cfg.mapper = compactClass(ccfg.getAffinityMapper());
+         cfg.partitions = aff.partitions();
+         cfg.partitionedBackups = ccfg.getBackups();
 -        cfg.dfltReplicas = dfltReplicas;
+         cfg.excludeNeighbors = excludeNeighbors;
  
          return cfg;
      }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9efcd803/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------


[02/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239


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

Branch: refs/heads/ignite-187
Commit: 5830deb20bf02705cb82203db074dfc2bb01473d
Parents: 3942853 4e7463d
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 16:04:40 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 16:04:40 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/IgniteFs.java   |   2 +-
 .../ignite/events/DiscoveryCustomEvent.java     |  56 --------
 .../org/apache/ignite/events/EventType.java     |  14 +-
 .../java/org/apache/ignite/igfs/package.html    |   2 +-
 .../internal/events/DiscoveryCustomEvent.java   |  68 +++++++++
 .../discovery/GridDiscoveryManager.java         |   7 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |  10 +-
 .../visor/node/VisorBasicConfiguration.java     |  17 ---
 .../node/VisorNodeEventsCollectorTask.java      |  10 +-
 .../internal/visor/node/VisorNodeGcTask.java    |  10 +-
 .../internal/visor/node/VisorNodePingTask.java  |  10 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   9 +-
 .../internal/GridDiscoveryEventSelfTest.java    |   9 +-
 ...dStartupWithUndefinedIgniteHomeSelfTest.java | 103 ++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 .../java/org/apache/ignite/igfs/package.html    |   2 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   7 +-
 .../commands/alert/VisorAlertCommand.scala      |   8 +-
 .../commands/cache/VisorCacheCommand.scala      |  82 +++++++----
 .../config/VisorConfigurationCommand.scala      | 140 ++++++++++---------
 .../commands/disco/VisorDiscoveryCommand.scala  |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  64 ++++++---
 .../commands/tasks/VisorTasksCommandSpec.scala  |   2 +-
 23 files changed, 392 insertions(+), 243 deletions(-)
----------------------------------------------------------------------



[09/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239


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

Branch: refs/heads/ignite-187
Commit: d1b9436a467148539d69a6dc70a084dd37ffa58f
Parents: 4f7dbf7 e1c0945
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 17:59:43 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 17:59:43 2015 +0300

----------------------------------------------------------------------
 config/ignite-log4j.xml                         |    2 +-
 .../datagrid/CacheContinuousQueryExample.java   |    2 +-
 ipc/shmem/Makefile.am                           |   15 +
 ipc/shmem/igniteshmem/Makefile.am               |   15 +
 ipc/shmem/include/Makefile.am                   |   15 +
 modules/clients/src/test/keystore/generate.sh   |   15 +-
 .../ignite/cache/query/ContinuousQuery.java     |   18 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |    3 +
 .../processors/cache/IgniteCacheProxy.java      |    6 +-
 .../optimized-classnames.previous.properties    |   15 +
 .../optimized/optimized-classnames.properties   | 1565 +-----------------
 .../TcpDiscoveryCustomEventMessage.java         |    3 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 modules/extdata/p2p/pom.xml                     |    6 -
 modules/hibernate/pom.xml                       |    6 -
 modules/indexing/pom.xml                        |    6 -
 modules/jta/pom.xml                             |    6 -
 modules/scalar/pom.xml                          |    6 -
 modules/spring/pom.xml                          |    6 -
 modules/visor-console/pom.xml                   |    7 -
 modules/web/pom.xml                             |    6 -
 modules/winservice/IgniteService.sln            |    2 +-
 .../IgniteService/IgniteService.csproj          |    2 +-
 .../config/benchmark-atomic-win.properties      |   15 +
 .../config/benchmark-atomic.properties          |   15 +
 .../config/benchmark-compute-win.properties     |   15 +
 .../config/benchmark-compute.properties         |   15 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-query-win.properties       |   15 +
 .../yardstick/config/benchmark-query.properties |   15 +
 .../config/benchmark-tx-win.properties          |   15 +
 .../yardstick/config/benchmark-tx.properties    |   15 +
 .../yardstick/config/benchmark-win.properties   |   15 +
 modules/yardstick/config/benchmark.properties   |   15 +
 pom.xml                                         |  150 +-
 36 files changed, 358 insertions(+), 1694 deletions(-)
----------------------------------------------------------------------



[10/50] incubator-ignite git commit: #ignite-239: review.

Posted by ak...@apache.org.
#ignite-239: 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/4ff4fd1d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4ff4fd1d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4ff4fd1d

Branch: refs/heads/ignite-187
Commit: 4ff4fd1d7ad0d7691bb911a5453d69d1a5ac627f
Parents: d1b9436
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:14:15 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:14:15 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 77 ++++++++++----------
 1 file changed, 38 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ff4fd1d/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index ca3abe9..6b8b197 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1295,51 +1295,19 @@ public class IgnitionEx {
         private void start0(GridStartContext startCtx) throws IgniteCheckedException {
             assert grid == null : "Grid is already started: " + name;
 
-            IgniteConfiguration cfg = startCtx.config() != null ? startCtx.config() : new IgniteConfiguration();
-
-            String ggHome = cfg.getIgniteHome();
-
-            // Set Ignite home.
-            if (ggHome == null)
-                ggHome = U.getIgniteHome();
-            else
-                // If user provided IGNITE_HOME - set it as a system property.
-                U.setIgniteHome(ggHome);
+            // Set configuration URL, if any, into system property.
+            if (startCtx.configUrl() != null)
+                System.setProperty(IGNITE_CONFIG_URL, startCtx.configUrl().toString());
 
-            U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
+            IgniteConfiguration cfg = startCtx.config() != null ? startCtx.config() : new IgniteConfiguration();
 
             // Ensure invariant.
             // It's a bit dirty - but this is a result of late refactoring
             // and I don't want to reshuffle a lot of code.
             assert F.eq(name, cfg.getGridName());
 
-            // Set configuration URL, if any, into system property.
-            if (startCtx.configUrl() != null)
-                System.setProperty(IGNITE_CONFIG_URL, startCtx.configUrl().toString());
-
-            // Initialize factory's log.
-            UUID nodeId = cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID();
-
-            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
-
-            assert cfgLog != null;
-
-            cfgLog = new GridLoggerProxy(cfgLog, null, name, U.id8(nodeId));
-
-            log = cfgLog.getLogger(G.class);
-
-            // Check Ignite home folder (after log is available).
-            if (ggHome != null) {
-                File ggHomeFile = new File(ggHome);
-
-                if (!ggHomeFile.exists() || !ggHomeFile.isDirectory())
-                    throw new IgniteCheckedException("Invalid Ignite installation home folder: " + ggHome);
-            }
-
             IgniteConfiguration myCfg = initializeConfiguration(cfg);
 
-            myCfg.setGridLogger(cfgLog);
-
             // Validate segmentation configuration.
             GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
 
@@ -1521,9 +1489,41 @@ public class IgnitionEx {
          */
         private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg)
             throws IgniteCheckedException {
+            // Initialize factory's log.
+            UUID nodeId = cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID();
+
+            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
+
+            assert cfgLog != null;
+
+            cfgLog = new GridLoggerProxy(cfgLog, null, name, U.id8(nodeId));
+
+            log = cfgLog.getLogger(G.class);
+
+            String ggHome = cfg.getIgniteHome();
+
+            // Set Ignite home.
+            if (ggHome == null)
+                ggHome = U.getIgniteHome();
+            else
+                // If user provided IGNITE_HOME - set it as a system property.
+                U.setIgniteHome(ggHome);
+
+            U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
+
+            // Check Ignite home folder (after log is available).
+            if (ggHome != null) {
+                File ggHomeFile = new File(ggHome);
+
+                if (!ggHomeFile.exists() || !ggHomeFile.isDirectory())
+                    throw new IgniteCheckedException("Invalid Ignite installation home folder: " + ggHome);
+            }
+
             IgniteConfiguration myCfg = new IgniteConfiguration(cfg);
 
-            myCfg.setIgniteHome(U.getIgniteHome());
+            myCfg.setIgniteHome(ggHome);
+
+            myCfg.setGridLogger(cfgLog);
 
             // Local host.
             String locHost = IgniteSystemProperties.getString(IGNITE_LOCAL_HOST);
@@ -1606,8 +1606,7 @@ public class IgnitionEx {
             if (myCfg.getMBeanServer() == null)
                 myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
 
-            if (myCfg.getNodeId() == null)
-               myCfg.setNodeId(UUID.randomUUID());
+            myCfg.setNodeId(nodeId);
 
             if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null)
                 myCfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);


[45/50] incubator-ignite git commit: # IGNITE-380 Added snippet generation.

Posted by ak...@apache.org.
# IGNITE-380 Added snippet generation.


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

Branch: refs/heads/ignite-187
Commit: 7488eddd6e0224d18ecdd9b7fb51566e926ead5d
Parents: b2675bc
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 14:49:27 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 14:49:27 2015 +0700

----------------------------------------------------------------------
 .../ignite/cache/CacheTypeFieldMetadata.java    |   2 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |  18 +--
 .../ignite/schema/generator/PojoGenerator.java  |   3 +-
 .../schema/generator/SnippetGenerator.java      | 138 +++++++++++++++++++
 .../apache/ignite/schema/ui/SchemaLoadApp.java  |   2 +
 5 files changed, 151 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
index 0041f10..625873b 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeFieldMetadata.java
@@ -50,7 +50,7 @@ public class CacheTypeFieldMetadata {
      * @param javaName Field name in java object.
      * @param javaType Field java type.
      */
-    public CacheTypeFieldMetadata(String javaName, Class<?> javaType, String dbName, int dbType) {
+    public CacheTypeFieldMetadata(String dbName, int dbType, String javaName, Class<?> javaType) {
         this.dbName = dbName;
         this.dbType = dbType;
         this.javaName = javaName;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
index 1e502c9..f25d9f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
@@ -230,6 +230,15 @@ public class CacheTypeMetadata {
     }
 
     /**
+     * Sets value fields.
+     *
+     * @param valFields New value fields.
+     */
+    public void setValueFields(Collection<CacheTypeFieldMetadata> valFields) {
+        this.valFields = valFields;
+    }
+
+    /**
      * Gets query-enabled fields.
      *
      * @return Collection of fields available for query.
@@ -318,13 +327,4 @@ public class CacheTypeMetadata {
     public void setGroups(Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps) {
         this.grps = grps;
     }
-
-    /**
-     * Sets value fields.
-     *
-     * @param valFields New value fields.
-     */
-    public void setValueFields(Collection<CacheTypeFieldMetadata> valFields) {
-        this.valFields = valFields;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
index 7d12e2c..500aa9a 100644
--- a/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/PojoGenerator.java
@@ -401,8 +401,7 @@ public class PojoGenerator {
      * @throws IOException If failed to write generated code into file.
      */
     public static void generate(PojoDescriptor pojo, String outFolder, String pkg, boolean constructor,
-        boolean includeKeys, ConfirmCallable askOverwrite)
-        throws IOException {
+        boolean includeKeys, ConfirmCallable askOverwrite) throws IOException {
         File pkgFolder = new File(outFolder, pkg.replace('.', File.separatorChar));
 
         if (!pkgFolder.exists() && !pkgFolder.mkdirs())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java
new file mode 100644
index 0000000..e54c019
--- /dev/null
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/generator/SnippetGenerator.java
@@ -0,0 +1,138 @@
+/*
+ * 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.schema.generator;
+
+import org.apache.ignite.schema.model.*;
+import org.apache.ignite.schema.ui.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.schema.ui.MessageBox.Result.*;
+
+/**
+ * Cache configuration snippet generator.
+ */
+public class SnippetGenerator {
+    /**
+     * Add type fields.
+     *
+     * @param src Source code lines.
+     * @param owner Fields owner collection.
+     * @param fields Fields metadata.
+     */
+    private static void addFields(Collection<String> src, String owner, Collection<PojoField> fields) {
+        for (PojoField field : fields) {
+            String javaTypeName = field.javaTypeName();
+
+            if (javaTypeName.startsWith("java.lang."))
+                javaTypeName = javaTypeName.substring(10);
+
+            src.add(owner + ".add(new CacheTypeFieldMetadata(\"" + field.dbName() + "\", " +
+                "java.sql.Types." + field.dbTypeName() + ",\"" +
+                field.javaName() + "\", " + javaTypeName + ".class));");
+        }
+    }
+
+    /**
+     * Generate java snippet for cache configuration with JDBC store.
+     *
+     * @param pojos POJO descriptors.
+     * @param pkg Types package.
+     * @param includeKeys {@code true} if key fields should be included into value class.
+     * @param out File to output snippet.
+     * @param askOverwrite Callback to ask user to confirm file overwrite.
+     * @throws IOException If generation failed.
+     */
+    public static void generate(Collection<PojoDescriptor> pojos, String pkg, boolean includeKeys, File out,
+        ConfirmCallable askOverwrite) throws IOException {
+        if (out.exists()) {
+            MessageBox.Result choice = askOverwrite.confirm(out.getName());
+
+            if (CANCEL == choice)
+                throw new IllegalStateException("Java configuration snippet generation was canceled!");
+
+            if (NO == choice || NO_TO_ALL == choice)
+                return;
+        }
+
+        Collection<String> src = new ArrayList<>(256);
+
+        src.add("// Code snippet for cache configuration.");
+        src.add("");
+        src.add("IgniteConfiguration cfg = new IgniteConfiguration();");
+        src.add("");
+        src.add("CacheConfiguration ccfg = new CacheConfiguration<>();");
+        src.add("");
+        src.add("DataSource dataSource = null; // TODO: Create data source for your database.");
+        src.add("");
+        src.add("// Create store. ");
+        src.add("CacheJdbcPojoStore store = new CacheJdbcPojoStore();");
+        src.add("store.setDataSource(dataSource);");
+        src.add("");
+        src.add("// Create store factory. ");
+        src.add("ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory<>(store));");
+        src.add("");
+        src.add("// Configure cache to use store. ");
+        src.add("ccfg.setReadThrough(true);");
+        src.add("ccfg.setWriteThrough(true);");
+        src.add("");
+        src.add("cfg.setCacheConfiguration(ccfg);");
+        src.add("");
+        src.add("// Configure cache types. ");
+        src.add("Collection<CacheTypeMetadata> meta = new ArrayList<>();");
+        src.add("");
+
+        boolean first = true;
+
+        for (PojoDescriptor pojo : pojos) {
+            String tbl = pojo.table();
+
+            src.add("// " + tbl + ".");
+            src.add((first ? "CacheTypeMetadata " : "") +  "type = new CacheTypeMetadata();");
+            src.add("type.setDatabaseSchema(\"" + pojo.schema() + "\");");
+            src.add("type.setDatabaseTable(\"" + tbl + "\");");
+            src.add("type.setKeyType(\"" + pkg + "." + pojo.keyClassName() + "\");");
+            src.add("type.setValueType(\"" +  pkg + "." + pojo.valueClassName() + "\");");
+            src.add("");
+
+            src.add("// Key fields for " + tbl + ".");
+            src.add((first ? "Collection<CacheTypeFieldMetadata> " : "") + "keys = new ArrayList<>();");
+            addFields(src, "keys", pojo.valueFields(includeKeys));
+            src.add("type.setKeyFields(keys);");
+            src.add("");
+
+            src.add("// Value fields for " + tbl + ".");
+            src.add((first ? "Collection<CacheTypeFieldMetadata> " : "") + "vals = new ArrayList<>();");
+            addFields(src, "vals", pojo.valueFields(includeKeys));
+            src.add("type.setValueFields(vals);");
+            src.add("");
+
+            first = false;
+        }
+
+        src.add("// Start Ignite node.");
+        src.add("Ignition.start(cfg);");
+
+        // Write generated code to file.
+        try (Writer writer = new BufferedWriter(new FileWriter(out))) {
+            for (String line : src)
+                writer.write(line + '\n');
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7488eddd/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java b/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
index 9c88624..f43c5ca 100644
--- a/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
+++ b/modules/schema-load/src/main/java/org/apache/ignite/schema/ui/SchemaLoadApp.java
@@ -435,6 +435,8 @@ public class SchemaLoadApp extends Application {
                 if (singleXml)
                     XmlGenerator.generate(pkg, all, includeKeys, new File(outFolder, "Ignite.xml"), askOverwrite);
 
+                SnippetGenerator.generate(all, pkg, includeKeys, new File(outFolder, "Ignite.snippet"), askOverwrite);
+
                 perceptualDelay(started);
 
                 return null;


[41/50] incubator-ignite git commit: Merge branch 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into sprint-2

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


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

Branch: refs/heads/ignite-187
Commit: b2675bc3a6875c462e3d0aeb1478ee4d72039c44
Parents: 9a93198 3b309f2
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Mar 2 17:50:47 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Mar 2 17:50:47 2015 +0300

----------------------------------------------------------------------
 examples/pom-standalone.xml                       | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 pom.xml                                           | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[44/50] incubator-ignite git commit: #ignite-237: change client test.

Posted by ak...@apache.org.
#ignite-237: change client test.


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

Branch: refs/heads/ignite-187
Commit: f144a892e9747b297610f7d9d19c44f362dd4b1f
Parents: 9efcd80
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 19:29:55 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 19:29:55 2015 +0300

----------------------------------------------------------------------
 .../client/integration/ClientAbstractMultiNodeSelfTest.java        | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f144a892/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
index e73f06d..9257b71 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -403,8 +403,6 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
 
             if (affinity(cache).isPrimaryOrBackup(g.cluster().localNode(), key))
                 assertEquals("zzz", cache.localPeek(key, CachePeekMode.ONHEAP));
-            else
-                assertNull(cache.localPeek(key, CachePeekMode.ONHEAP));
         }
     }
 


[49/50] incubator-ignite git commit: # minor cleanup of unused code.

Posted by ak...@apache.org.
# minor cleanup of unused code.


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

Branch: refs/heads/ignite-187
Commit: 2a68e64eb50460b4709a7bc4b8e88be3db0604e8
Parents: d33abdb
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 16:25:25 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 16:25:25 2015 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheAffinityConfiguration.java      | 10 ----------
 .../ignite/visor/commands/cache/VisorCacheCommand.scala   |  1 -
 2 files changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2a68e64e/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 6ff2d2d..613b034 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -46,9 +46,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     /** Cache affinity partitions. */
     private Integer partitions;
 
-    /** Cache partitioned affinity default replicas. */
-    private Integer dfltReplicas;
-
     /** Cache partitioned affinity exclude neighbors. */
     private Boolean excludeNeighbors;
 
@@ -107,13 +104,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @return Cache partitioned affinity default replicas.
-     */
-    @Nullable public Integer defaultReplicas() {
-        return dfltReplicas;
-    }
-
-    /**
      * @return Cache partitioned affinity exclude neighbors.
      */
     @Nullable public Boolean excludeNeighbors() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2a68e64e/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 2db43dc..69da6f5 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -805,7 +805,6 @@ object VisorCacheCommand {
         cacheT += ("Affinity Function", safe(affinityCfg.function()))
         cacheT += ("Affinity Backups", affinityCfg.partitionedBackups())
         cacheT += ("Affinity Partitions", safe(affinityCfg.partitions()))
-        cacheT += ("Affinity Default Replicas", safe(affinityCfg.defaultReplicas()))
         cacheT += ("Affinity Exclude Neighbors", safe(affinityCfg.excludeNeighbors()))
         cacheT += ("Affinity Mapper", safe(affinityCfg.mapper()))
 


[46/50] incubator-ignite git commit: # IGNITE-298 Escape cache and grid names.

Posted by ak...@apache.org.
# IGNITE-298 Escape cache and grid names.


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

Branch: refs/heads/ignite-187
Commit: b81c6a25ba2ed743c251a34eb044123667d4be44
Parents: 7488edd
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 15:31:35 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 15:31:35 2015 +0700

----------------------------------------------------------------------
 .../commands/cache/VisorCacheCommand.scala      | 17 +++++++-----
 .../commands/cache/VisorCacheScanCommand.scala  |  7 +++--
 .../config/VisorConfigurationCommand.scala      |  5 ++--
 .../visor/commands/node/VisorNodeCommand.scala  |  6 ++---
 .../scala/org/apache/ignite/visor/visor.scala   | 27 +++++++++++---------
 5 files changed, 35 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index e2ca05b..2db43dc 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.ClusterNode
 import org.apache.ignite.internal.util.typedef._
 import org.apache.ignite.internal.visor.cache._
 import org.apache.ignite.internal.visor.node.{VisorGridConfiguration, VisorNodeConfigurationCollectorTask}
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.lang.IgniteBiTuple
 import org.jetbrains.annotations._
 
@@ -243,10 +244,11 @@ class VisorCacheCommand {
                 }
 
                 val cacheName = argValue("c", argLst) match {
-                    case Some("<default>") | Some(CACHE_DFLT) =>
+                    case Some(dfltName) if dfltName == DFLT_CACHE_KEY || dfltName == DFLT_CACHE_NAME =>
                         argLst = argLst.filter(_._1 != "c") ++ Seq("c" -> null)
 
                         Some(null)
+
                     case cn => cn
                 }
 
@@ -422,9 +424,9 @@ class VisorCacheCommand {
      */
     private def mkCacheName(@Nullable s: String): String = {
         if (s == null) {
-            val v = mfind(CACHE_DFLT)
+            val v = mfind(DFLT_CACHE_KEY)
 
-            "<default>" + (if (v.isDefined) "(@" + v.get._1 + ')' else "")
+            DFLT_CACHE_NAME + (if (v.isDefined) "(@" + v.get._1 + ')' else "")
         }
         else {
             val v = mfind(s)
@@ -438,7 +440,7 @@ class VisorCacheCommand {
      *
      * @param s Cache host.
      */
-    private def registerCacheName(@Nullable s: String) = setVarIfAbsent(if (s != null) s else CACHE_DFLT, "c")
+    private def registerCacheName(@Nullable s: String) = setVarIfAbsent(if (s != null) s else DFLT_CACHE_KEY, "c")
 
     /**
      * ===Command===
@@ -744,8 +746,11 @@ object VisorCacheCommand {
         ref = VisorConsoleCommand(cmd.cache, cmd.cache)
     )
 
+    /** Default cache name to show on screen. */
+    private final val DFLT_CACHE_NAME = escapeName(null)
+    
     /** Default cache key. */
-    protected val CACHE_DFLT = "<default>-" + UUID.randomUUID().toString
+    protected val DFLT_CACHE_KEY = DFLT_CACHE_NAME + "-" + UUID.randomUUID().toString
 
     /** Singleton command */
     private val cmd = new VisorCacheCommand
@@ -852,7 +857,7 @@ object VisorCacheCommand {
         cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
         cacheT += ("Memory Mode", cfg.memoryMode())
         cacheT += ("Keep Values Bytes", cfg.valueBytes())
-        cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())
+        cacheT += ("Off-Heap Size", if (cfg.offsetHeapMaxMemory() >= 0) cfg.offsetHeapMaxMemory() else NA)
 
         cacheT += ("Loader Factory Class Name", safe(cfg.loaderFactory()))
         cacheT += ("Writer Factory Class Name", safe(cfg.writerFactory()))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
index a3d6967..4fb1896 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.visor.query.VisorQueryTask.VisorQueryArg
 import org.apache.ignite.internal.visor.query.{VisorQueryNextPageTask, VisorQueryResult, VisorQueryTask}
 
 import org.apache.ignite.cluster.ClusterNode
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.lang.IgniteBiTuple
 
 import org.apache.ignite.visor.commands._
@@ -177,18 +178,16 @@ class VisorCacheScanCommand {
                     return
             }
 
-        def escapeCacheName(name: String) = if (name == null) "<default>" else name
-
         var res: VisorQueryResult = fullRes
 
         if (res.rows.isEmpty) {
-            println("Cache: " + escapeCacheName(cacheName) + " is empty")
+            println("Cache: " + escapeName(cacheName) + " is empty")
 
             return
         }
 
         def render() {
-            println("Entries in cache: " + escapeCacheName(cacheName))
+            println("Entries in cache: " + escapeName(cacheName))
 
             val t = VisorTextTable()
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
index a2ab512..788ef19 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/config/VisorConfigurationCommand.scala
@@ -21,6 +21,7 @@ import org.apache.ignite._
 import org.apache.ignite.cluster.ClusterNode
 import org.apache.ignite.internal.util.{IgniteUtils => U}
 import org.apache.ignite.internal.visor.node.VisorNodeConfigurationCollectorTask
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.apache.ignite.lang.IgniteBiTuple
 
 import java.lang.System._
@@ -199,7 +200,7 @@ class VisorConfigurationCommand {
 
             val basic = cfg.basic()
 
-            cmnT += ("Grid name", safe(basic.gridName(), "<default>"))
+            cmnT += ("Grid name", escapeName(basic.gridName()))
             cmnT += ("Ignite home", safe(basic.ggHome()))
             cmnT += ("Localhost", safe(basic.localHost()))
             cmnT += ("Node ID", safe(basic.nodeId()))
@@ -407,7 +408,7 @@ class VisorConfigurationCommand {
                 println("\nNo system properties defined.")
 
             cfg.caches().foreach(cacheCfg => {
-                VisorCacheCommand.showCacheConfiguration("\nCache '" + safe(cacheCfg.name()) + "':", cacheCfg)
+                VisorCacheCommand.showCacheConfiguration("\nCache '" + escapeName(cacheCfg.name()) + "':", cacheCfg)
             })
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
index 79625e2..eeb2a6c 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/node/VisorNodeCommand.scala
@@ -22,7 +22,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes._
 import org.apache.ignite.internal.util.lang.{GridFunc => F}
 import org.apache.ignite.internal.util.typedef.X
 import org.apache.ignite.internal.util.{IgniteUtils => U}
-
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 import org.jetbrains.annotations._
 
 import java.util.UUID
@@ -190,7 +190,7 @@ class VisorNodeCommand {
                             t += ("JRE information", node.attribute(ATTR_JIT_NAME))
                             t += ("Non-loopback IPs", node.attribute(ATTR_IPS))
                             t += ("Enabled MACs", node.attribute(ATTR_MACS))
-                            t += ("Grid name", safe(gridName, "<default>"))
+                            t += ("Grid name", escapeName(gridName))
                             t += ("JVM start time", formatDateTime(m.getStartTime))
                             t += ("Node start time", formatDateTime(m.getNodeStartTime))
                             t += ("Up time", X.timeSpan2HMSM(m.getUpTime))
@@ -245,7 +245,7 @@ class VisorNodeCommand {
                             t += ("Language runtime", node.attribute(ATTR_LANG_RUNTIME))
                             t += ("Ignite version", verStr)
                             t += ("JRE information", node.attribute(ATTR_JIT_NAME))
-                            t += ("Grid name", safe(gridName, "<default>"))
+                            t += ("Grid name", escapeName(gridName))
                             t += ("JVM start time", formatDateTime(m.getStartTime))
                             t += ("Node start time", formatDateTime(m.getNodeStartTime))
                             t += ("Up time", X.timeSpan2HMSM(m.getUpTime))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b81c6a25/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index b9b5dbc..964de1d 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -17,36 +17,39 @@
 
 package org.apache.ignite.visor
 
-import java.io._
-import java.net._
-import java.text._
-import java.util.concurrent._
-import java.util.{HashSet => JHashSet, _}
-
 import org.apache.ignite.IgniteSystemProperties._
+import org.apache.ignite._
 import org.apache.ignite.cluster.{ClusterGroup, ClusterMetrics, ClusterNode}
 import org.apache.ignite.configuration.IgniteConfiguration
 import org.apache.ignite.events.EventType._
 import org.apache.ignite.events.{DiscoveryEvent, Event}
 import org.apache.ignite.internal.IgniteComponentType._
 import org.apache.ignite.internal.IgniteNodeAttributes._
+import org.apache.ignite.internal.IgniteVersionUtils._
 import org.apache.ignite.internal.cluster.ClusterGroupEmptyCheckedException
 import org.apache.ignite.internal.processors.spring.IgniteSpringProcessor
-import org.apache.ignite.internal.{IgniteVersionUtils, IgniteEx}
-import IgniteVersionUtils._
 import org.apache.ignite.internal.util.lang.{GridFunc => F}
 import org.apache.ignite.internal.util.typedef._
-import org.apache.ignite.internal.util.{IgniteUtils => U, GridConfigurationFinder}
+import org.apache.ignite.internal.util.{GridConfigurationFinder, IgniteUtils => U}
 import org.apache.ignite.internal.visor.VisorTaskArgument
 import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTask
 import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTask.VisorNodeEventsCollectorTaskArg
+import org.apache.ignite.internal.visor.util.VisorTaskUtils._
+import org.apache.ignite.internal.IgniteEx
 import org.apache.ignite.lang.{IgniteNotPeerDeployable, IgnitePredicate}
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi
 import org.apache.ignite.thread.IgniteThreadPoolExecutor
-import org.apache.ignite.visor.commands.{VisorConsoleCommand, VisorTextTable}
-import org.apache.ignite._
+
 import org.jetbrains.annotations.Nullable
 
+import java.io._
+import java.net._
+import java.text._
+import java.util.concurrent._
+import java.util.{HashSet => JHashSet, _}
+
+import org.apache.ignite.visor.commands.{VisorConsoleCommand, VisorTextTable}
+
 import scala.collection.JavaConversions._
 import scala.collection.immutable
 import scala.io.StdIn
@@ -1317,7 +1320,7 @@ object visor extends VisorTag {
             else {
                 val n = ignite.name
 
-                if (n == null) "<default>" else n
+                escapeName(n)
             }
         )
         t += ("Config path", safe(cfgPath))


[04/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-187
Commit: bcf802d9a235b0a0e451bd2ac50ec0369e02f76b
Parents: 148ca09 dd66167
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 27 16:56:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 27 16:56:52 2015 +0300

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |  12 +
 .../src/main/java/org/apache/ignite/Ignite.java |   4 +-
 .../configuration/QueryConfiguration.java       |  37 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |   6 +-
 .../config/GridTestProperties.java              |  10 +-
 .../client/hadoop/GridHadoopClientProtocol.java |   6 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     | 412 ++++++++++++++++++
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    |   3 +-
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |   3 +-
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |  10 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     | 413 -------------------
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |   2 +-
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |   4 +-
 .../hadoop/GridHadoopClassLoader.java           |  12 +-
 .../processors/hadoop/GridHadoopSetup.java      |   8 +-
 .../processors/hadoop/GridHadoopUtils.java      |   4 +-
 .../collections/GridHadoopHashMultimapBase.java |   2 +-
 .../GridHadoopExternalCommunication.java        |  14 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |   6 +-
 .../v2/GridHadoopV2JobResourceManager.java      |   2 +-
 .../GridHadoopClientProtocolSelfTest.java       |   6 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   2 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java |   2 +-
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |   2 +-
 .../IgfsHadoopFileSystemAbstractSelfTest.java   |   1 +
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |   2 +-
 .../hadoop/GridHadoopGroupingTest.java          |   4 +-
 .../igfs/IgfsPerformanceBenchmark.java          |   9 +-
 28 files changed, 530 insertions(+), 468 deletions(-)
----------------------------------------------------------------------



[38/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/sprint-2' into sprint-2

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/sprint-2' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: 028e44637fe9ff3fe3347d685f7d52422ff1cb0d
Parents: 2511ebe 235758c
Author: anovikov <an...@gridgain.com>
Authored: Mon Mar 2 21:44:49 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Mon Mar 2 21:44:49 2015 +0700

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/IgniteCluster.java  | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[31/50] incubator-ignite git commit: #ignite-334: Change IgniteConfiguration.pluginConfigurations to array.

Posted by ak...@apache.org.
#ignite-334: Change IgniteConfiguration.pluginConfigurations to array.


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

Branch: refs/heads/ignite-187
Commit: 4df7c85990391206c7e6247658267185f11f86a9
Parents: c847e88
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 13:44:40 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 13:44:40 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/configuration/IgniteConfiguration.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4df7c859/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 72ab928..54404c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -311,7 +311,7 @@ public class IgniteConfiguration {
     private TransactionConfiguration txCfg = new TransactionConfiguration();
 
     /** */
-    private Collection<? extends PluginConfiguration> pluginCfgs;
+    private PluginConfiguration[] pluginCfgs;
 
     /** Flag indicating whether cache sanity check is enabled. */
     private boolean cacheSanityCheckEnabled = DFLT_CACHE_SANITY_CHECK_ENABLED;
@@ -1874,14 +1874,14 @@ public class IgniteConfiguration {
     /**
      * @return Plugin configurations.
      */
-    public Collection<? extends PluginConfiguration> getPluginConfigurations() {
+    public PluginConfiguration[] getPluginConfigurations() {
         return pluginCfgs;
     }
 
     /**
      * @param pluginCfgs Plugin configurations.
      */
-    public void setPluginConfigurations(Collection<? extends PluginConfiguration> pluginCfgs) {
+    public void setPluginConfigurations(PluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;
     }
 


[15/50] incubator-ignite git commit: #ignite-237: move CacheConsistentHashAffinityFunction.

Posted by ak...@apache.org.
#ignite-237: move CacheConsistentHashAffinityFunction.


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

Branch: refs/heads/ignite-187
Commit: e3782644789c565162870fb2a41bd753e7631922
Parents: 414410b
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 19:36:34 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 19:36:34 2015 +0300

----------------------------------------------------------------------
 .../CacheConsistentHashAffinityFunction.java    | 703 -------------------
 1 file changed, 703 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e3782644/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java b/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
deleted file mode 100644
index 35be9b8..0000000
--- a/modules/core/src/main/java/org/gridgain/benchmarks/risk/affinity/CacheConsistentHashAffinityFunction.java
+++ /dev/null
@@ -1,703 +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.gridgain.benchmarks.risk.affinity;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Affinity function for partitioned cache. This function supports the following
- * configuration:
- * <ul>
- * <li>
- *      {@code backups} - Use this flag to control how many back up nodes will be
- *      assigned to every key. The default value is {@code 0}.
- * </li>
- * <li>
- *      {@code replicas} - Generally the more replicas a node gets, the more key assignments
- *      it will receive. You can configure different number of replicas for a node by
- *      setting user attribute with name {@link #getReplicaCountAttributeName()} to some
- *      number. Default value is {@code 512} defined by {@link #DFLT_REPLICA_COUNT} constant.
- * </li>
- * <li>
- *      {@code backupFilter} - Optional filter for back up nodes. If provided, then only
- *      nodes that pass this filter will be selected as backup nodes. If not provided, then
- *      primary and backup nodes will be selected out of all nodes available for this cache.
- * </li>
- * </ul>
- * <p>
- * Cache affinity can be configured for individual caches via {@link org.apache.ignite.configuration.CacheConfiguration#getAffinity()} method.
- */
-@Deprecated
-public class CacheConsistentHashAffinityFunction implements CacheAffinityFunction {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Flag to enable/disable consistency check (for internal use only). */
-    private static final boolean AFFINITY_CONSISTENCY_CHECK = Boolean.getBoolean("IGNITE_AFFINITY_CONSISTENCY_CHECK");
-
-    /** Default number of partitions. */
-    public static final int DFLT_PARTITION_COUNT = 10000;
-
-    /** Default replica count for partitioned caches. */
-    public static final int DFLT_REPLICA_COUNT = 128;
-
-    /**
-     * Name of node attribute to specify number of replicas for a node.
-     * Default value is {@code gg:affinity:node:replicas}.
-     */
-    public static final String DFLT_REPLICA_COUNT_ATTR_NAME = "gg:affinity:node:replicas";
-
-    /** Node hash. */
-    private transient GridConsistentHash<NodeInfo> nodeHash;
-
-    /** Total number of partitions. */
-    private int parts = DFLT_PARTITION_COUNT;
-
-    /** */
-    private int replicas = DFLT_REPLICA_COUNT;
-
-    /** */
-    private String attrName = DFLT_REPLICA_COUNT_ATTR_NAME;
-
-    /** */
-    private boolean exclNeighbors;
-
-    /**
-     * Optional backup filter. First node passed to this filter is primary node,
-     * and second node is a node being tested.
-     */
-    private IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter;
-
-    /** */
-    private CacheAffinityNodeHashResolver hashIdRslvr = new CacheAffinityNodeAddressHashResolver();
-
-    /** Injected grid. */
-    @IgniteInstanceResource
-    private Ignite ignite;
-
-    /** Injected cache name. */
-    @CacheNameResource
-    private String cacheName;
-
-    /** Injected logger. */
-    @LoggerResource
-    private IgniteLogger log;
-
-    /** Initialization flag. */
-    @SuppressWarnings("TransientFieldNotInitialized")
-    private transient AtomicBoolean init = new AtomicBoolean();
-
-    /** Latch for initializing. */
-    @SuppressWarnings({"TransientFieldNotInitialized"})
-    private transient CountDownLatch initLatch = new CountDownLatch(1);
-
-    /** Nodes IDs. */
-    @GridToStringInclude
-    @SuppressWarnings({"TransientFieldNotInitialized"})
-    private transient ConcurrentMap<UUID, NodeInfo> addedNodes = new ConcurrentHashMap<>();
-
-    /** Optional backup filter. */
-    @GridToStringExclude
-    private final IgniteBiPredicate<NodeInfo, NodeInfo> backupIdFilter = new IgniteBiPredicate<NodeInfo, NodeInfo>() {
-        @Override public boolean apply(NodeInfo primaryNodeInfo, NodeInfo nodeInfo) {
-            return backupFilter == null || backupFilter.apply(primaryNodeInfo.node(), nodeInfo.node());
-        }
-    };
-
-    /** Map of neighbors. */
-    @SuppressWarnings("TransientFieldNotInitialized")
-    private transient ConcurrentMap<UUID, Collection<UUID>> neighbors =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Empty constructor with all defaults.
-     */
-    public CacheConsistentHashAffinityFunction() {
-        // No-op.
-    }
-
-    /**
-     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other
-     * and specified number of backups.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
-     *      of each other.
-     */
-    public CacheConsistentHashAffinityFunction(boolean exclNeighbors) {
-        this.exclNeighbors = exclNeighbors;
-    }
-
-    /**
-     * Initializes affinity with flag to exclude same-host-neighbors from being backups of each other,
-     * and specified number of backups and partitions.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups
-     *      of each other.
-     * @param parts Total number of partitions.
-     */
-    public CacheConsistentHashAffinityFunction(boolean exclNeighbors, int parts) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.exclNeighbors = exclNeighbors;
-        this.parts = parts;
-    }
-
-    /**
-     * Initializes optional counts for replicas and backups.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @param parts Total number of partitions.
-     * @param backupFilter Optional back up filter for nodes. If provided, backups will be selected
-     *      from all nodes that pass this filter. First argument for this filter is primary node, and second
-     *      argument is node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     */
-    public CacheConsistentHashAffinityFunction(int parts,
-                                               @Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        A.ensure(parts != 0, "parts != 0");
-
-        this.parts = parts;
-        this.backupFilter = backupFilter;
-    }
-
-    /**
-     * Gets default count of virtual replicas in consistent hash ring.
-     * <p>
-     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName()}
-     * name will be checked first. If it is absent, then this value will be used.
-     *
-     * @return Count of virtual replicas in consistent hash ring.
-     */
-    public int getDefaultReplicas() {
-        return replicas;
-    }
-
-    /**
-     * Sets default count of virtual replicas in consistent hash ring.
-     * <p>
-     * To determine node replicas, node attribute with {@link #getReplicaCountAttributeName} name
-     * will be checked first. If it is absent, then this value will be used.
-     *
-     * @param replicas Count of virtual replicas in consistent hash ring.s
-     */
-    public void setDefaultReplicas(int replicas) {
-        this.replicas = replicas;
-    }
-
-    /**
-     * Gets total number of key partitions. To ensure that all partitions are
-     * equally distributed across all nodes, please make sure that this
-     * number is significantly larger than a number of nodes. Also, partition
-     * size should be relatively small. Try to avoid having partitions with more
-     * than quarter million keys.
-     * <p>
-     * Note that for fully replicated caches this method should always
-     * return {@code 1}.
-     *
-     * @return Total partition count.
-     */
-    public int getPartitions() {
-        return parts;
-    }
-
-    /**
-     * Sets total number of partitions.
-     *
-     * @param parts Total number of partitions.
-     */
-    public void setPartitions(int parts) {
-        this.parts = parts;
-    }
-
-    /**
-     * Gets hash ID resolver for nodes. This resolver is used to provide
-     * alternate hash ID, other than node ID.
-     * <p>
-     * Node IDs constantly change when nodes get restarted, which causes them to
-     * be placed on different locations in the hash ring, and hence causing
-     * repartitioning. Providing an alternate hash ID, which survives node restarts,
-     * puts node on the same location on the hash ring, hence minimizing required
-     * repartitioning.
-     *
-     * @return Hash ID resolver.
-     */
-    public CacheAffinityNodeHashResolver getHashIdResolver() {
-        return hashIdRslvr;
-    }
-
-    /**
-     * Sets hash ID resolver for nodes. This resolver is used to provide
-     * alternate hash ID, other than node ID.
-     * <p>
-     * Node IDs constantly change when nodes get restarted, which causes them to
-     * be placed on different locations in the hash ring, and hence causing
-     * repartitioning. Providing an alternate hash ID, which survives node restarts,
-     * puts node on the same location on the hash ring, hence minimizing required
-     * repartitioning.
-     *
-     * @param hashIdRslvr Hash ID resolver.
-     */
-    public void setHashIdResolver(CacheAffinityNodeHashResolver hashIdRslvr) {
-        this.hashIdRslvr = hashIdRslvr;
-    }
-
-    /**
-     * Gets optional backup filter. If not {@code null}, backups will be selected
-     * from all nodes that pass this filter. First node passed to this filter is primary node,
-     * and second node is a node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @return Optional backup filter.
-     */
-    @Nullable public IgniteBiPredicate<ClusterNode, ClusterNode> getBackupFilter() {
-        return backupFilter;
-    }
-
-    /**
-     * Sets optional backup filter. If provided, then backups will be selected from all
-     * nodes that pass this filter. First node being passed to this filter is primary node,
-     * and second node is a node being tested.
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code backupFilter} is set.
-     *
-     * @param backupFilter Optional backup filter.
-     */
-    public void setBackupFilter(@Nullable IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter) {
-        this.backupFilter = backupFilter;
-    }
-
-    /**
-     * Gets optional attribute name for replica count. If not provided, the
-     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
-     *
-     * @return User attribute name for replica count for a node.
-     */
-    public String getReplicaCountAttributeName() {
-        return attrName;
-    }
-
-    /**
-     * Sets optional attribute name for replica count. If not provided, the
-     * default is {@link #DFLT_REPLICA_COUNT_ATTR_NAME}.
-     *
-     * @param attrName User attribute name for replica count for a node.
-     */
-    public void setReplicaCountAttributeName(String attrName) {
-        this.attrName = attrName;
-    }
-
-    /**
-     * Checks flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @return {@code True} if nodes residing on the same host may not act as backups of each other.
-     */
-    public boolean isExcludeNeighbors() {
-        return exclNeighbors;
-    }
-
-    /**
-     * Sets flag to exclude same-host-neighbors from being backups of each other (default is {@code false}).
-     * <p>
-     * Note that {@code excludeNeighbors} parameter is ignored if {@code #getBackupFilter()} is set.
-     *
-     * @param exclNeighbors {@code True} if nodes residing on the same host may not act as backups of each other.
-     */
-    public void setExcludeNeighbors(boolean exclNeighbors) {
-        this.exclNeighbors = exclNeighbors;
-    }
-
-    /**
-     * Gets neighbors for a node.
-     *
-     * @param node Node.
-     * @return Neighbors.
-     */
-    private Collection<UUID> neighbors(final ClusterNode node) {
-        Collection<UUID> ns = neighbors.get(node.id());
-
-        if (ns == null) {
-            Collection<ClusterNode> nodes = ignite.cluster().forHost(node).nodes();
-
-            ns = F.addIfAbsent(neighbors, node.id(), new ArrayList<>(F.nodeIds(nodes)));
-        }
-
-        return ns;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public List<List<ClusterNode>> assignPartitions(CacheAffinityFunctionContext ctx) {
-        List<List<ClusterNode>> res = new ArrayList<>(parts);
-
-        Collection<ClusterNode> topSnapshot = ctx.currentTopologySnapshot();
-
-        for (int part = 0; part < parts; part++) {
-            res.add(F.isEmpty(topSnapshot) ?
-                Collections.<ClusterNode>emptyList() :
-                // Wrap affinity nodes with unmodifiable list since unmodifiable generic collection
-                // doesn't provide equals and hashCode implementations.
-                U.sealList(nodes(part, topSnapshot, ctx.backups())));
-        }
-
-        return res;
-    }
-
-    /**
-     * Assigns nodes to one partition.
-     *
-     * @param part Partition to assign nodes for.
-     * @param nodes Cache topology nodes.
-     * @return Assigned nodes, first node is primary, others are backups.
-     */
-    public Collection<ClusterNode> nodes(int part, Collection<ClusterNode> nodes, int backups) {
-        if (nodes == null)
-            return Collections.emptyList();
-
-        int nodesSize = nodes.size();
-
-        if (nodesSize == 0)
-            return Collections.emptyList();
-
-        if (nodesSize == 1) // Minor optimization.
-            return nodes;
-
-        initialize();
-
-        final Map<NodeInfo, ClusterNode> lookup = new GridLeanMap<>(nodesSize);
-
-        // Store nodes in map for fast lookup.
-        for (ClusterNode n : nodes)
-            // Add nodes into hash circle, if absent.
-            lookup.put(resolveNodeInfo(n), n);
-
-        Collection<NodeInfo> selected;
-
-        if (backupFilter != null) {
-            final IgnitePredicate<NodeInfo> p = new P1<NodeInfo>() {
-                @Override public boolean apply(NodeInfo id) {
-                    return lookup.containsKey(id);
-                }
-            };
-
-            final NodeInfo primaryId = nodeHash.node(part, p);
-
-            IgnitePredicate<NodeInfo> backupPrimaryIdFilter = new IgnitePredicate<NodeInfo>() {
-                @Override public boolean apply(NodeInfo node) {
-                    return backupIdFilter.apply(primaryId, node);
-                }
-            };
-
-            Collection<NodeInfo> backupIds = nodeHash.nodes(part, backups, p, backupPrimaryIdFilter);
-
-            if (F.isEmpty(backupIds) && primaryId != null) {
-                ClusterNode n = lookup.get(primaryId);
-
-                assert n != null;
-
-                return Collections.singletonList(n);
-            }
-
-            selected = primaryId != null ? F.concat(false, primaryId, backupIds) : backupIds;
-        }
-        else {
-            if (!exclNeighbors) {
-                selected = nodeHash.nodes(part, backups == Integer.MAX_VALUE ? backups : backups + 1, new P1<NodeInfo>() {
-                    @Override public boolean apply(NodeInfo id) {
-                        return lookup.containsKey(id);
-                    }
-                });
-
-                if (selected.size() == 1) {
-                    NodeInfo id = F.first(selected);
-
-                    assert id != null : "Node ID cannot be null in affinity node ID collection: " + selected;
-
-                    ClusterNode n = lookup.get(id);
-
-                    assert n != null;
-
-                    return Collections.singletonList(n);
-                }
-            }
-            else {
-                int primaryAndBackups = backups + 1;
-
-                selected = new ArrayList<>(primaryAndBackups);
-
-                final Collection<NodeInfo> selected0 = selected;
-
-                List<NodeInfo> ids = nodeHash.nodes(part, primaryAndBackups, new P1<NodeInfo>() {
-                    @Override public boolean apply(NodeInfo id) {
-                        ClusterNode n = lookup.get(id);
-
-                        if (n == null)
-                            return false;
-
-                        Collection<UUID> neighbors = neighbors(n);
-
-                        for (NodeInfo id0 : selected0) {
-                            ClusterNode n0 = lookup.get(id0);
-
-                            if (n0 == null)
-                                return false;
-
-                            Collection<UUID> neighbors0 = neighbors(n0);
-
-                            if (F.containsAny(neighbors0, neighbors))
-                                return false;
-                        }
-
-                        selected0.add(id);
-
-                        return true;
-                    }
-                });
-
-                if (AFFINITY_CONSISTENCY_CHECK)
-                    assert F.eqOrdered(ids, selected);
-            }
-        }
-
-        Collection<ClusterNode> ret = new ArrayList<>(selected.size());
-
-        for (NodeInfo id : selected) {
-            ClusterNode n = lookup.get(id);
-
-            assert n != null;
-
-            ret.add(n);
-        }
-
-        return ret;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key) {
-        initialize();
-
-        return U.safeAbs(key.hashCode() % parts);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partitions() {
-        initialize();
-
-        return parts;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        addedNodes = new ConcurrentHashMap<>();
-        neighbors = new ConcurrentHashMap8<>();
-
-        initLatch = new CountDownLatch(1);
-
-        init = new AtomicBoolean();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeNode(UUID nodeId) {
-        NodeInfo info = addedNodes.remove(nodeId);
-
-        if (info == null)
-            return;
-
-        nodeHash.removeNode(info);
-
-        neighbors.clear();
-    }
-
-    /**
-     * Resolve node info for specified node.
-     * Add node to hash circle if this is the first node invocation.
-     *
-     * @param n Node to get info for.
-     * @return Node info.
-     */
-    private NodeInfo resolveNodeInfo(ClusterNode n) {
-        UUID nodeId = n.id();
-        NodeInfo nodeInfo = addedNodes.get(nodeId);
-
-        if (nodeInfo != null)
-            return nodeInfo;
-
-        assert hashIdRslvr != null;
-
-        nodeInfo = new NodeInfo(nodeId, hashIdRslvr.resolve(n), n);
-
-        neighbors.clear();
-
-        nodeHash.addNode(nodeInfo, replicas(n));
-
-        addedNodes.put(nodeId, nodeInfo);
-
-        return nodeInfo;
-    }
-
-    /** {@inheritDoc} */
-    private void initialize() {
-        if (!init.get() && init.compareAndSet(false, true)) {
-            if (log.isInfoEnabled())
-                log.info("Consistent hash configuration [cacheName=" + cacheName + ", partitions=" + parts +
-                    ", excludeNeighbors=" + exclNeighbors + ", replicas=" + replicas +
-                    ", backupFilter=" + backupFilter + ", hashIdRslvr=" + hashIdRslvr + ']');
-
-            nodeHash = new GridConsistentHash<>();
-
-            initLatch.countDown();
-        }
-        else {
-            if (initLatch.getCount() > 0) {
-                try {
-                    U.await(initLatch);
-                }
-                catch (IgniteInterruptedCheckedException ignored) {
-                    // Recover interrupted state flag.
-                    Thread.currentThread().interrupt();
-                }
-            }
-        }
-    }
-
-    /**
-     * @param n Node.
-     * @return Replicas.
-     */
-    private int replicas(ClusterNode n) {
-        Integer nodeReplicas = n.attribute(attrName);
-
-        if (nodeReplicas == null)
-            nodeReplicas = replicas;
-
-        return nodeReplicas;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(CacheConsistentHashAffinityFunction.class, this);
-    }
-
-    /**
-     * Node hash ID.
-     */
-    private static final class NodeInfo implements Comparable<NodeInfo> {
-        /** Node ID. */
-        private UUID nodeId;
-
-        /** Hash ID. */
-        private Object hashId;
-
-        /** Grid node. */
-        private ClusterNode node;
-
-        /**
-         * @param nodeId Node ID.
-         * @param hashId Hash ID.
-         * @param node Rich node.
-         */
-        private NodeInfo(UUID nodeId, Object hashId, ClusterNode node) {
-            assert nodeId != null;
-            assert hashId != null;
-
-            this.hashId = hashId;
-            this.nodeId = nodeId;
-            this.node = node;
-        }
-
-        /**
-         * @return Node ID.
-         */
-        public UUID nodeId() {
-            return nodeId;
-        }
-
-        /**
-         * @return Hash ID.
-         */
-        public Object hashId() {
-            return hashId;
-        }
-
-        /**
-         * @return Node.
-         */
-        public ClusterNode node() {
-            return node;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return hashId.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            if (!(obj instanceof NodeInfo))
-                return false;
-
-            NodeInfo that = (NodeInfo)obj;
-
-            // If objects are equal, hash codes should be the same.
-            // Cannot use that.hashId.equals(hashId) due to Comparable<N> interface restrictions.
-            return that.nodeId.equals(nodeId) && that.hashCode() == hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compareTo(NodeInfo o) {
-            int diff = nodeId.compareTo(o.nodeId);
-
-            if (diff == 0) {
-                int h1 = hashCode();
-                int h2 = o.hashCode();
-
-                diff = h1 == h2 ? 0 : (h1 < h2 ? -1 : 1);
-            }
-
-            return diff;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(NodeInfo.class, this);
-        }
-    }
-}


[05/50] incubator-ignite git commit: # ignite-237

Posted by ak...@apache.org.
# ignite-237


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

Branch: refs/heads/ignite-187
Commit: e50d1eadc11651c9029b81ccd0636ea003f5b1a7
Parents: bcf802d
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 27 17:12:25 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 27 17:12:25 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/GridCacheAffinityBackupsSelfTest.java |  8 --------
 .../GridCacheContinuousQueryAbstractSelfTest.java       | 12 ++++++------
 2 files changed, 6 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e50d1ead/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
index 88a3cc1..0758993 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridCacheAffinityBackupsSelfTest.java
@@ -58,14 +58,6 @@ public class GridCacheAffinityBackupsSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testConsistentHashBackups() throws Exception {
-        for (int i = 0; i < nodesCnt; i++)
-            checkBackups(i, new CacheRendezvousAffinityFunction());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testRendezvousBackups() throws Exception {
         for (int i = 0; i < nodesCnt; i++)
             checkBackups(i, new CacheRendezvousAffinityFunction());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e50d1ead/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 0d9e189..d2b1786 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -165,12 +165,14 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             Set<Cache.Entry<Object, Object>> entries = ((IgniteKernal) grid(i)).cache(null).entrySet();
 
             for (Cache.Entry entry : entries) {
-                info("Not removed entry " + grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey()));
-                info("Not removed entry " + grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey()));
+                boolean primary = grid(i).affinity(null).isPrimary(grid(i).localNode(), entry.getKey());
+
+                boolean backup = grid(i).affinity(null).isBackup(grid(i).localNode(), entry.getKey());
+
+                info("Not removed entry [grid=" + i + ", primary=" + primary + ", backup=" + backup + ']');
+
                 allEmpty = false;
             }
-
-            info("Cache is not empty: " + ((IgniteKernal) grid(i)).cache(null).entrySet());
         }
 
         assertTrue(allEmpty);
@@ -844,8 +846,6 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         finally {
             stopGrid("anotherGrid");
         }
-
-        awaitPartitionMapExchange();
     }
 
     /**


[39/50] incubator-ignite git commit: Merge branch 'ignite-136' into sprint-2

Posted by ak...@apache.org.
Merge branch 'ignite-136' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: 9a931983f51f96ad9b96551aee654874e746aa94
Parents: 028e446 2e115bf
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Mar 2 17:47:24 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Mar 2 17:47:24 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |   9 +
 .../distributed/dht/GridDhtCacheEntry.java      |   3 +
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +
 .../GridCacheAbstractLocalStoreSelfTest.java    | 440 +++++++++++++++++++
 .../GridCachePartitionedLocalStoreSelfTest.java |  51 +++
 ...chePartitionedOffHeapLocalStoreSelfTest.java |  56 +++
 .../GridCacheReplicatedLocalStoreSelfTest.java  |  51 +++
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |  51 +++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 9 files changed, 670 insertions(+)
----------------------------------------------------------------------



[37/50] incubator-ignite git commit: # MInor JavADoc change.

Posted by ak...@apache.org.
# MInor JavADoc change.


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

Branch: refs/heads/ignite-187
Commit: 235758cd98f4aca320a6f9c07c5d90ddb956d796
Parents: b0e6ab1
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 2 17:44:16 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 2 17:44:16 2015 +0300

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/IgniteCluster.java  | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/235758cd/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index 8669559..c24ba5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -28,10 +28,10 @@ import java.util.*;
 import java.util.concurrent.*;
 
 /**
- * {@code GridCluster} also provides a handle on {@link #nodeLocalMap()} which provides map-like functionality
- * linked to current grid node. Node-local map is useful for saving shared state between job executions
- * on the grid. Additionally you can also ping, start, and restart remote nodes, map keys to caching nodes,
- * and get other useful information about topology.
+ * Represents whole cluster (all available nodes) and also provides a handle on {@link #nodeLocalMap()} which
+ * provides map-like functionality linked to current grid node. Node-local map is useful for saving shared state
+ * between job executions on the grid. Additionally you can also ping, start, and restart remote nodes, map keys to
+ * caching nodes, and get other useful information about topology.
  */
 public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
     /**


[06/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239


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

Branch: refs/heads/ignite-187
Commit: 1689a203b0dca44b2435e676d5c2c9c9c3fda095
Parents: 4f0feee dd66167
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 27 17:16:36 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 27 17:16:36 2015 +0300

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |  12 +
 .../src/main/java/org/apache/ignite/Ignite.java |   4 +-
 .../configuration/QueryConfiguration.java       |  37 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |   6 +-
 .../config/GridTestProperties.java              |  10 +-
 .../client/hadoop/GridHadoopClientProtocol.java |   6 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     | 412 ++++++++++++++++++
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    |   3 +-
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |   3 +-
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |  10 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     | 413 -------------------
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |   2 +-
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |   4 +-
 .../hadoop/GridHadoopClassLoader.java           |  12 +-
 .../processors/hadoop/GridHadoopSetup.java      |   8 +-
 .../processors/hadoop/GridHadoopUtils.java      |   4 +-
 .../collections/GridHadoopHashMultimapBase.java |   2 +-
 .../GridHadoopExternalCommunication.java        |  14 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |   6 +-
 .../v2/GridHadoopV2JobResourceManager.java      |   2 +-
 .../GridHadoopClientProtocolSelfTest.java       |   6 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   2 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java |   2 +-
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |   2 +-
 .../IgfsHadoopFileSystemAbstractSelfTest.java   |   1 +
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |   2 +-
 .../hadoop/GridHadoopGroupingTest.java          |   4 +-
 .../igfs/IgfsPerformanceBenchmark.java          |   9 +-
 28 files changed, 530 insertions(+), 468 deletions(-)
----------------------------------------------------------------------



[08/50] incubator-ignite git commit: IGNITE-136 Added test eviction for local store.

Posted by ak...@apache.org.
IGNITE-136 Added test eviction for local store.


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

Branch: refs/heads/ignite-187
Commit: 2e115bf663aef70f6141119adcbd11e2c7bbbaaa
Parents: 3a466e8
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri Feb 27 17:57:04 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Fri Feb 27 17:57:04 2015 +0300

----------------------------------------------------------------------
 .../GridCacheAbstractLocalStoreSelfTest.java    | 30 ++++++++++++++++++++
 1 file changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2e115bf6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
index 781c6a6..c43b8f8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.processors.cache.store.*;
 import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -35,6 +36,7 @@ import org.jetbrains.annotations.*;
 
 import javax.cache.*;
 import javax.cache.configuration.*;
+import javax.cache.expiry.*;
 import javax.cache.integration.*;
 import java.util.*;
 import java.util.concurrent.*;
@@ -170,6 +172,34 @@ public abstract class GridCacheAbstractLocalStoreSelfTest extends GridCommonAbst
     /**
      * @throws Exception If failed.
      */
+    public void testEvict() throws Exception {
+        Ignite ignite1 = startGrid(1);
+
+        IgniteCache<Object, Object> cache = ignite1.jcache(null).withExpiryPolicy(new CreatedExpiryPolicy(
+            new Duration(TimeUnit.MILLISECONDS, 100L)));
+
+        // Putting entry.
+        for (int i = 0; i < KEYS; i++)
+            cache.put(i, i);
+
+        // Wait when entry 
+        U.sleep(200);
+
+        // Check that entry is evicted from cache, but local store does contain it.
+        for (int i = 0; i < KEYS; i++) {
+            cache.localEvict(Arrays.asList(i));
+
+            assertNull(cache.localPeek(i));
+
+            assertEquals(i, (int)LOCAL_STORE_1.load(i).get1());
+
+            assertEquals(i, cache.get(i));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testPrimaryNode() throws Exception {
         Ignite ignite1 = startGrid(1);
 


[18/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-187
Commit: 10ac88a701618249f4e23ecb467fe0099c034a4e
Parents: 8e58738 a792c99
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 20:03:24 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 20:03:24 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/GridProperties.java  | 78 -------------------
 .../ignite/internal/GridUpdateNotifier.java     |  2 +-
 .../apache/ignite/internal/IgniteKernal.java    | 18 ++---
 .../ignite/internal/IgniteProperties.java       | 79 ++++++++++++++++++++
 .../ignite/internal/IgniteVersionUtils.java     |  8 +-
 .../plugin/IgnitePluginProcessor.java           | 24 ++++++
 .../apache/ignite/plugin/PluginProvider.java    |  5 ++
 .../internal/GridUpdateNotifierSelfTest.java    |  2 +-
 8 files changed, 123 insertions(+), 93 deletions(-)
----------------------------------------------------------------------



[07/50] incubator-ignite git commit: # ignite-239

Posted by ak...@apache.org.
# ignite-239


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

Branch: refs/heads/ignite-187
Commit: 4f7dbf76ea6638792abd23e7d0327eabd8bc377b
Parents: 1689a20
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 27 17:27:18 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Feb 27 17:27:18 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 71 +++++++++-----------
 1 file changed, 33 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f7dbf76/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index f39cd7e..ca3abe9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1517,36 +1517,28 @@ public class IgnitionEx {
         /**
          * @param cfg Ignite configuration copy to.
          * @return New ignite configuration.
+         * @throws IgniteCheckedException If failed.
          */
         private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg)
             throws IgniteCheckedException {
             IgniteConfiguration myCfg = new IgniteConfiguration(cfg);
 
-            initializeDefaultConfigurationParameters(myCfg);
-
-            return myCfg;
-        }
-
-        /**
-         * Initialize default parameters.
-         */
-        public void initializeDefaultConfigurationParameters(IgniteConfiguration cfg) throws IgniteCheckedException {
-            cfg.setIgniteHome(U.getIgniteHome());
+            myCfg.setIgniteHome(U.getIgniteHome());
 
             // Local host.
             String locHost = IgniteSystemProperties.getString(IGNITE_LOCAL_HOST);
 
-            cfg.setLocalHost(F.isEmpty(locHost) ? cfg.getLocalHost() : locHost);
+            myCfg.setLocalHost(F.isEmpty(locHost) ? myCfg.getLocalHost() : locHost);
 
             // Override daemon flag if it was set on the factory.
             if (daemon)
-                cfg.setDaemon(true);
+                myCfg.setDaemon(true);
 
-            Marshaller marsh = cfg.getMarshaller();
+            Marshaller marsh = myCfg.getMarshaller();
 
             if (marsh == null) {
                 if (!U.isHotSpot()) {
-                    U.warn(log, "GridOptimizedMarshaller is not supported on this JVM " +
+                    U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
                             "(only Java HotSpot VMs are supported). Switching to standard JDK marshalling - " +
                             "object serialization performance will be significantly slower.",
                         "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
@@ -1554,7 +1546,7 @@ public class IgnitionEx {
                     marsh = new JdkMarshaller();
                 }
                 else if (!OptimizedMarshaller.available()) {
-                    U.warn(log, "GridOptimizedMarshaller is not supported on this JVM " +
+                    U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
                             "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
                             "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +
                             "Switching to standard JDK marshalling - " +
@@ -1567,7 +1559,7 @@ public class IgnitionEx {
                     marsh = new OptimizedMarshaller();
             }
             else if (marsh instanceof OptimizedMarshaller && !U.isHotSpot()) {
-                U.warn(log, "Using GridOptimizedMarshaller on untested JVM (only Java HotSpot VMs were tested) - " +
+                U.warn(log, "Using OptimizedMarshaller on untested JVM (only Java HotSpot VMs were tested) - " +
                         "object serialization behavior could yield unexpected results.",
                     "Using GridOptimizedMarshaller on untested JVM.");
             }
@@ -1576,7 +1568,7 @@ public class IgnitionEx {
             String depModeName = IgniteSystemProperties.getString(IGNITE_DEP_MODE_OVERRIDE);
 
             if (!F.isEmpty(depModeName)) {
-                if (!F.isEmpty(cfg.getCacheConfiguration())) {
+                if (!F.isEmpty(myCfg.getCacheConfiguration())) {
                     U.quietAndInfo(log, "Skipping deployment mode override for caches (custom closure " +
                         "execution may not work for console Visor)");
                 }
@@ -1584,8 +1576,8 @@ public class IgnitionEx {
                     try {
                         DeploymentMode depMode = DeploymentMode.valueOf(depModeName);
 
-                        if (cfg.getDeploymentMode() != depMode)
-                            cfg.setDeploymentMode(depMode);
+                        if (myCfg.getDeploymentMode() != depMode)
+                            myCfg.setDeploymentMode(depMode);
                     }
                     catch (IllegalArgumentException e) {
                         throw new IgniteCheckedException("Failed to override deployment mode using system property " +
@@ -1595,12 +1587,12 @@ public class IgnitionEx {
                 }
             }
 
-            cfg.setMarshaller(marsh);
+            myCfg.setMarshaller(marsh);
 
-            cfg.setConnectorConfiguration(cfg.getConnectorConfiguration() != null ?
-                new ConnectorConfiguration(cfg.getConnectorConfiguration()) : null);
+            myCfg.setConnectorConfiguration(myCfg.getConnectorConfiguration() != null ?
+                new ConnectorConfiguration(myCfg.getConnectorConfiguration()) : null);
 
-            IgfsConfiguration[] igfsCfgs = cfg.getIgfsConfiguration();
+            IgfsConfiguration[] igfsCfgs = myCfg.getIgfsConfiguration();
 
             if (igfsCfgs != null) {
                 IgfsConfiguration[] clone = igfsCfgs.clone();
@@ -1608,19 +1600,19 @@ public class IgnitionEx {
                 for (int i = 0; i < igfsCfgs.length; i++)
                     clone[i] = new IgfsConfiguration(igfsCfgs[i]);
 
-                cfg.setIgfsConfiguration(clone);
+                myCfg.setIgfsConfiguration(clone);
             }
 
-            if (cfg.getMBeanServer() == null)
-                cfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
+            if (myCfg.getMBeanServer() == null)
+                myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
 
-            if (cfg.getNodeId() == null)
-               cfg.setNodeId(UUID.randomUUID());
+            if (myCfg.getNodeId() == null)
+               myCfg.setNodeId(UUID.randomUUID());
 
-            if (cfg.getPeerClassLoadingLocalClassPathExclude() == null)
-                cfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);
+            if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null)
+                myCfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);
 
-            StreamerConfiguration[] streamerCfgs = cfg.getStreamerConfiguration();
+            StreamerConfiguration[] streamerCfgs = myCfg.getStreamerConfiguration();
 
             if (streamerCfgs != null) {
                 StreamerConfiguration[] clone = streamerCfgs.clone();
@@ -1628,26 +1620,29 @@ public class IgnitionEx {
                 for (int i = 0; i < streamerCfgs.length; i++)
                     clone[i] = new StreamerConfiguration(streamerCfgs[i]);
 
-                cfg.setStreamerConfiguration(clone);
+                myCfg.setStreamerConfiguration(clone);
             }
 
-            cfg.setTransactionConfiguration(cfg.getTransactionConfiguration() != null ?
-                new TransactionConfiguration(cfg.getTransactionConfiguration()) : null);
+            myCfg.setTransactionConfiguration(myCfg.getTransactionConfiguration() != null ?
+                new TransactionConfiguration(myCfg.getTransactionConfiguration()) : null);
 
-            if (cfg.getUserAttributes() == null) {
+            if (myCfg.getUserAttributes() == null) {
                 Map<String, ?> emptyAttr = Collections.emptyMap();
-                cfg.setUserAttributes(emptyAttr);
+                myCfg.setUserAttributes(emptyAttr);
             }
 
-            initializeDefaultCacheConfiguration(cfg);
+            initializeDefaultCacheConfiguration(myCfg);
+
+            initializeDefaultSpi(myCfg);
 
-            initializeDefaultSpi(cfg);
+            return myCfg;
         }
 
         /**
          * Initialize default cache configuration.
          *
          * @param cfg Ignite configuration.
+         * @throws IgniteCheckedException If failed.
          */
         public void initializeDefaultCacheConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
             CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();


[40/50] incubator-ignite git commit: Merge branch 'ignite-382' into sprint-2

Posted by ak...@apache.org.
Merge branch 'ignite-382' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: 3b309f250e872fff4ea290fb2b8acb7388fadfb2
Parents: 028e446 f252ee4
Author: anovikov <an...@gridgain.com>
Authored: Mon Mar 2 21:48:07 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Mon Mar 2 21:48:07 2015 +0700

----------------------------------------------------------------------
 examples/pom-standalone.xml                       | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 pom.xml                                           | 4 ++--
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[34/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/sprint-2' into sprint-2

Posted by ak...@apache.org.
Merge remote-tracking branch 'origin/sprint-2' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: 403d60feed477a4598509b619b2f63de307b9b5b
Parents: 8baca04 9671471
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Mar 2 17:16:40 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Mar 2 17:16:40 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |   5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++++----------
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 3 files changed, 267 insertions(+), 331 deletions(-)
----------------------------------------------------------------------



[33/50] incubator-ignite git commit: ignite-342 review

Posted by ak...@apache.org.
ignite-342 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/8baca046
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8baca046
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8baca046

Branch: refs/heads/ignite-187
Commit: 8baca046e02e703b5ce5c0a2f1a34ba279392a3e
Parents: 11efb91
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Mar 2 17:16:26 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Mar 2 17:16:26 2015 +0300

----------------------------------------------------------------------
 .../affinity/GridAffinityAssignmentCache.java   |  13 +-
 .../processors/cache/GridCacheContext.java      |  19 ++
 .../processors/cache/GridCacheProcessor.java    |  15 ++
 ...acheAbstractUsersAffinityMapperSelfTest.java | 207 +++++++++++++++++++
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |  45 ++++
 ...heReplicatedUsersAffinityMapperSelfTest.java |  45 ++++
 .../GridCacheTxUsersAffinityMapperSelfTest.java |  45 ++++
 7 files changed, 388 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 9c12a17..69795b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -305,7 +305,18 @@ public class GridAffinityAssignmentCache {
             }
         }
 
-        return aff.partition(affMapper.affinityKey(key));
+        return aff.partition(affinityKey(key));
+    }
+
+    /**
+     * If Key is {@link GridCacheInternal GridCacheInternal} entry when won't passed into user's mapper and
+     * will use {@link GridCacheDefaultAffinityKeyMapper default}.
+     *
+     * @param key Key.
+     * @return Affinity key.
+     */
+    private Object affinityKey(Object key) {
+        return (key instanceof GridCacheInternal ? ctx.defaultAffMapper() : affMapper).affinityKey(key);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 3ec013c..44f8e69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -164,6 +165,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Cached local rich node. */
     private ClusterNode locNode;
 
+    /** Default cache affinity mapper. */
+    private CacheAffinityKeyMapper affMapper;
+
     /**
      * Thread local projection. If it's set it means that method call was initiated
      * by child projection of initial cache.
@@ -1016,6 +1020,20 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Default affinity key mapper.
+     */
+    public CacheAffinityKeyMapper defaultAffMapper() {
+        return affMapper;
+    }
+
+    /**
+     * Sets default affinity key mapper.
+     */
+    public void defaultAffMapper(CacheAffinityKeyMapper dfltAffMapper) {
+        this.affMapper = dfltAffMapper;
+    }
+
+    /**
      * @param p Single predicate.
      * @return Array containing single predicate.
      */
@@ -1770,6 +1788,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         evictMgr = null;
         qryMgr = null;
         dataStructuresMgr = null;
+        affMapper = null;
 
         mgrs.clear();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index e99c706..f74f969 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -618,6 +618,17 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             U.startLifecycleAware(lifecycleAwares(cfg, jta.tmLookup(), cfgStore));
 
+            // Init default key mapper.
+            CacheAffinityKeyMapper dfltAffMapper;
+
+            if (cfg.getAffinityMapper().getClass().equals(GridCacheDefaultAffinityKeyMapper.class))
+                dfltAffMapper = cfg.getAffinityMapper();
+            else {
+                dfltAffMapper = new GridCacheDefaultAffinityKeyMapper();
+
+                prepare(cfg, dfltAffMapper, false);
+            }
+
             cfgs[i] = cfg; // Replace original configuration value.
 
             GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
@@ -655,6 +666,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 drMgr,
                 jta);
 
+            cacheCtx.defaultAffMapper(dfltAffMapper);
+
             GridCacheAdapter cache = null;
 
             switch (cfg.getCacheMode()) {
@@ -793,6 +806,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     drMgr,
                     jta);
 
+                cacheCtx.defaultAffMapper(dfltAffMapper);
+
                 GridDhtCacheAdapter dht = null;
 
                 switch (cfg.getAtomicityMode()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractUsersAffinityMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractUsersAffinityMapperSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractUsersAffinityMapperSelfTest.java
new file mode 100644
index 0000000..71f28ce
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractUsersAffinityMapperSelfTest.java
@@ -0,0 +1,207 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+
+/**
+ * Test affinity mapper.
+ */
+public abstract class GridCacheAbstractUsersAffinityMapperSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final int KEY_CNT = 1000;
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    public static final CacheAffinityKeyMapper AFFINITY_MAPPER = new UsersAffinityKeyMapper();
+
+    /** */
+    public GridCacheAbstractUsersAffinityMapperSelfTest() {
+        super(false /* doesn't start grid */);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setName(null);
+        cacheCfg.setCacheMode(getCacheMode());
+        cacheCfg.setAtomicityMode(getAtomicMode());
+        cacheCfg.setDistributionMode(getDistributionMode());
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setPreloadMode(CachePreloadMode.SYNC);
+        cacheCfg.setAffinityMapper(AFFINITY_MAPPER);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(spi);
+
+        return cfg;
+    }
+
+    /**
+     * @return Distribution mode.
+     */
+    protected abstract CacheDistributionMode getDistributionMode();
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode getAtomicMode();
+
+    /**
+     * @return Cache mode.
+     */
+    protected abstract CacheMode getCacheMode();
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityMapper() throws Exception {
+        IgniteCache<Object, Object> cache = startGrid(0).jcache(null);
+
+        for (int i = 0; i < KEY_CNT; i++) {
+            cache.put(String.valueOf(i), String.valueOf(i));
+
+            cache.put(new TestAffinityKey(i, String.valueOf(i)), i);
+        }
+
+        assertEquals(1, cache.get(new TestAffinityKey(1, "1")));
+
+        startGrid(1);
+
+        for (int i = 0; i < KEY_CNT; i++)
+            grid(i % 2).compute().affinityRun(null, new TestAffinityKey(1, "1"), new NoopClosure());
+    }
+
+    /**
+     * Test key for field annotation.
+     */
+    private static class TestAffinityKey implements Externalizable {
+        /** Key. */
+        private int key;
+
+        /** Affinity key. */
+        @CacheAffinityKeyMapped
+        private String affKey;
+
+        /**
+         * Constructor.
+         */
+        public TestAffinityKey() {
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param key Key.
+         * @param affKey Affinity key.
+         */
+        TestAffinityKey(int key, String affKey) {
+            this.key = key;
+            this.affKey = affKey;
+        }
+
+        /**
+         * @return Key.
+         */
+        public int key() {
+            return key;
+        }
+
+        /**
+         * @return Affinity key.
+         */
+        public String affinityKey() {
+            return affKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return o instanceof TestAffinityKey && key == ((TestAffinityKey)o).key();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return key + affKey.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(key);
+            out.writeUTF(affKey);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            key = in.readInt();
+            affKey = in.readUTF();
+        }
+    }
+
+    /**
+     * Users affinity mapper.
+     */
+    private static class UsersAffinityKeyMapper extends GridCacheDefaultAffinityKeyMapper{
+        /** {@inheritDoc} */
+        @Override public Object affinityKey(Object key) {
+            GridArgumentCheck.notNull(key, "key");
+
+            assertFalse("GridCacheInternal entry mustn't be passed in user's key mapper.",
+                key instanceof GridCacheInternal);
+
+            return super.affinityKey(key);
+        }
+    }
+
+    /**
+     * Noop closure.
+     */
+    private static class NoopClosure implements IgniteRunnable {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicUsersAffinityMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicUsersAffinityMapperSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicUsersAffinityMapperSelfTest.java
new file mode 100644
index 0000000..8a80e35
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicUsersAffinityMapperSelfTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+/**
+ * Test affinity mapper.
+ */
+public class GridCacheAtomicUsersAffinityMapperSelfTest extends GridCacheAbstractUsersAffinityMapperSelfTest {
+    /** */
+    public GridCacheAtomicUsersAffinityMapperSelfTest() {
+        super();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode getDistributionMode() {
+        return CacheDistributionMode.PARTITIONED_ONLY;
+    };
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode getAtomicMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode getCacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedUsersAffinityMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedUsersAffinityMapperSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedUsersAffinityMapperSelfTest.java
new file mode 100644
index 0000000..47e5dc7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedUsersAffinityMapperSelfTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+/**
+ * Test affinity mapper.
+ */
+public class GridCacheReplicatedUsersAffinityMapperSelfTest extends GridCacheAbstractUsersAffinityMapperSelfTest {
+    /** */
+    public GridCacheReplicatedUsersAffinityMapperSelfTest() {
+        super();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode getDistributionMode() {
+        return CacheDistributionMode.PARTITIONED_ONLY;
+    };
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode getAtomicMode() {
+        return CacheAtomicityMode.ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode getCacheMode() {
+        return CacheMode.REPLICATED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8baca046/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTxUsersAffinityMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTxUsersAffinityMapperSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTxUsersAffinityMapperSelfTest.java
new file mode 100644
index 0000000..61af04e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTxUsersAffinityMapperSelfTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+
+/**
+ * Test affinity mapper.
+ */
+public class GridCacheTxUsersAffinityMapperSelfTest extends GridCacheAbstractUsersAffinityMapperSelfTest {
+    /** */
+    public GridCacheTxUsersAffinityMapperSelfTest() {
+        super();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode getDistributionMode() {
+        return CacheDistributionMode.PARTITIONED_ONLY;
+    };
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode getAtomicMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode getCacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+}


[21/50] incubator-ignite git commit: #ignite-239: small refactoring.

Posted by ak...@apache.org.
#ignite-239: small refactoring.


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

Branch: refs/heads/ignite-187
Commit: 0dfe349c28e2fc24b3b2e092f4f36c1d63f84ff3
Parents: 55669d4
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 21:21:11 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 21:21:11 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 28 ++++++++++----------
 1 file changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0dfe349c/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 9770205..c2c23f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1393,7 +1393,7 @@ public class IgnitionEx {
                 Class helperCls = Class.forName("org.apache.ignite.util.GridConfigurationHelper");
 
                 helperCls.getMethod("overrideConfiguration", IgniteConfiguration.class, Properties.class,
-                        String.class, IgniteLogger.class).invoke(helperCls, myCfg, System.getProperties(), name, log);
+                    String.class, IgniteLogger.class).invoke(helperCls, myCfg, System.getProperties(), name, log);
             }
             catch (Exception ignored) {
                 // No-op.
@@ -1522,14 +1522,14 @@ public class IgnitionEx {
             // for correct segment after segmentation happens.
             if (!F.isEmpty(cfg.getSegmentationResolvers()) && segPlc == RESTART_JVM && !cfg.isWaitForSegmentOnStart()) {
                 U.warn(log, "Found potential configuration problem (forgot to enable waiting for segment" +
-                        "on start?) [segPlc=" + segPlc + ", wait=false]");
+                    "on start?) [segPlc=" + segPlc + ", wait=false]");
             }
 
             myCfg.setTransactionConfiguration(myCfg.getTransactionConfiguration() != null ?
-                    new TransactionConfiguration(myCfg.getTransactionConfiguration()) : null);
+                new TransactionConfiguration(myCfg.getTransactionConfiguration()) : null);
 
             myCfg.setConnectorConfiguration(myCfg.getConnectorConfiguration() != null ?
-                    new ConnectorConfiguration(myCfg.getConnectorConfiguration()) : null);
+                new ConnectorConfiguration(myCfg.getConnectorConfiguration()) : null);
 
             // Local host.
             String locHost = IgniteSystemProperties.getString(IGNITE_LOCAL_HOST);
@@ -1546,7 +1546,7 @@ public class IgnitionEx {
             if (!F.isEmpty(depModeName)) {
                 if (!F.isEmpty(myCfg.getCacheConfiguration())) {
                     U.quietAndInfo(log, "Skipping deployment mode override for caches (custom closure " +
-                            "execution may not work for console Visor)");
+                        "execution may not work for console Visor)");
                 }
                 else {
                     try {
@@ -1557,8 +1557,8 @@ public class IgnitionEx {
                     }
                     catch (IllegalArgumentException e) {
                         throw new IgniteCheckedException("Failed to override deployment mode using system property " +
-                                "(are there any misspellings?)" +
-                                "[name=" + IGNITE_DEP_MODE_OVERRIDE + ", value=" + depModeName + ']', e);
+                            "(are there any misspellings?)" +
+                            "[name=" + IGNITE_DEP_MODE_OVERRIDE + ", value=" + depModeName + ']', e);
                     }
                 }
             }
@@ -1576,18 +1576,18 @@ public class IgnitionEx {
             if (marsh == null) {
                 if (!U.isHotSpot()) {
                     U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
-                            "(only Java HotSpot VMs are supported). Switching to standard JDK marshalling - " +
-                            "object serialization performance will be significantly slower.",
+                        "(only Java HotSpot VMs are supported). Switching to standard JDK marshalling - " +
+                        "object serialization performance will be significantly slower.",
                         "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
 
                     marsh = new JdkMarshaller();
                 }
                 else if (!OptimizedMarshaller.available()) {
                     U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
-                            "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
-                            "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +
-                            "Switching to standard JDK marshalling - " +
-                            "object serialization performance will be significantly slower.",
+                        "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
+                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +
+                        "Switching to standard JDK marshalling - " +
+                        "object serialization performance will be significantly slower.",
                         "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
 
                     marsh = new JdkMarshaller();
@@ -1597,7 +1597,7 @@ public class IgnitionEx {
             }
             else if (marsh instanceof OptimizedMarshaller && !U.isHotSpot()) {
                 U.warn(log, "Using OptimizedMarshaller on untested JVM (only Java HotSpot VMs were tested) - " +
-                        "object serialization behavior could yield unexpected results.",
+                    "object serialization behavior could yield unexpected results.",
                     "Using GridOptimizedMarshaller on untested JVM.");
             }
 


[19/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-239


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

Branch: refs/heads/ignite-187
Commit: ba010da403a0425f6d0ccfe006e41e1997bbbf9f
Parents: 8476463 036bd71
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 15:26:27 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 15:26:27 2015 +0300

----------------------------------------------------------------------
 .../store/CacheNodeWithStoreStartup.java        |   6 +-
 .../store/jdbc/CacheJdbcPersonStore.java        | 115 +++++++------------
 .../apache/ignite/cache/store/CacheStore.java   |   4 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |   2 +-
 .../ignite/cache/store/CacheStoreSession.java   |  17 ++-
 .../apache/ignite/internal/GridProperties.java  |  78 -------------
 .../ignite/internal/GridUpdateNotifier.java     |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  18 +--
 .../ignite/internal/IgniteProperties.java       |  79 +++++++++++++
 .../ignite/internal/IgniteVersionUtils.java     |   8 +-
 .../processors/cache/GridCacheStoreManager.java |   6 +-
 .../plugin/IgnitePluginProcessor.java           |  24 ++++
 .../apache/ignite/plugin/PluginProvider.java    |   5 +
 .../internal/GridUpdateNotifierSelfTest.java    |   2 +-
 .../junits/cache/TestCacheSession.java          |   5 +
 .../cache/TestThreadLocalCacheSession.java      |   5 +
 .../HibernateReadWriteAccessStrategy.java       |  81 ++++++++++---
 17 files changed, 264 insertions(+), 193 deletions(-)
----------------------------------------------------------------------



[22/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-187
Commit: b378307eaef5e41a5de0cff33c7265cec9729d61
Parents: f27e9d2 036bd71
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 21:22:32 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 21:22:32 2015 +0300

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |   12 +
 config/ignite-log4j.xml                         |    2 +-
 .../ComputeFibonacciContinuationExample.java    |   15 +-
 .../datagrid/CacheContinuousQueryExample.java   |    2 +-
 .../store/CacheNodeWithStoreStartup.java        |    6 +-
 .../store/jdbc/CacheJdbcPersonStore.java        |  115 +-
 .../examples/ScalarContinuationExample.scala    |   15 +-
 ipc/shmem/Makefile.am                           |   15 +
 ipc/shmem/igniteshmem/Makefile.am               |   15 +
 ipc/shmem/include/Makefile.am                   |   15 +
 modules/clients/src/test/keystore/generate.sh   |   15 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    4 +-
 .../main/java/org/apache/ignite/IgniteFs.java   |    2 +-
 .../ignite/cache/query/ContinuousQuery.java     |   18 +-
 .../apache/ignite/cache/store/CacheStore.java   |    4 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |    2 +-
 .../ignite/cache/store/CacheStoreSession.java   |   17 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    2 +-
 .../configuration/QueryConfiguration.java       |   37 +-
 .../ignite/events/DiscoveryCustomEvent.java     |   56 -
 .../org/apache/ignite/events/EventType.java     |   14 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |    6 +-
 .../java/org/apache/ignite/igfs/package.html    |    2 +-
 .../apache/ignite/internal/GridProperties.java  |   78 -
 .../ignite/internal/GridUpdateNotifier.java     |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   18 +-
 .../ignite/internal/IgniteProperties.java       |   79 +
 .../ignite/internal/IgniteVersionUtils.java     |    8 +-
 .../internal/events/DiscoveryCustomEvent.java   |   71 +
 .../discovery/GridDiscoveryManager.java         |    7 +-
 .../processors/cache/GridCacheStoreManager.java |    6 +-
 .../processors/cache/IgniteCacheProxy.java      |    6 +-
 .../plugin/IgnitePluginProcessor.java           |   24 +
 .../cache/VisorCacheMetricsCollectorTask.java   |   10 +-
 .../visor/node/VisorBasicConfiguration.java     |   17 -
 .../node/VisorNodeEventsCollectorTask.java      |   10 +-
 .../internal/visor/node/VisorNodeGcTask.java    |   10 +-
 .../internal/visor/node/VisorNodePingTask.java  |   10 +-
 .../optimized-classnames.previous.properties    |   15 +
 .../optimized/optimized-classnames.properties   | 1565 +-----------------
 .../apache/ignite/plugin/PluginProvider.java    |    5 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    9 +-
 .../TcpDiscoveryCustomEventMessage.java         |    3 +
 .../internal/GridDiscoveryEventSelfTest.java    |    9 +-
 .../internal/GridUpdateNotifierSelfTest.java    |    2 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 ...dStartupWithUndefinedIgniteHomeSelfTest.java |  103 ++
 .../config/GridTestProperties.java              |   10 +-
 .../junits/cache/TestCacheSession.java          |    5 +
 .../cache/TestThreadLocalCacheSession.java      |    5 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    1 +
 modules/extdata/p2p/pom.xml                     |    6 -
 modules/hadoop/pom.xml                          |   10 -
 .../client/hadoop/GridHadoopClientProtocol.java |    6 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  412 +++++
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    |    3 +-
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |    3 +-
 .../java/org/apache/ignite/igfs/package.html    |    2 +-
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |   10 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  413 -----
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |    2 +-
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |    4 +-
 .../hadoop/GridHadoopClassLoader.java           |   12 +-
 .../processors/hadoop/GridHadoopSetup.java      |    8 +-
 .../processors/hadoop/GridHadoopUtils.java      |    4 +-
 .../collections/GridHadoopHashMultimapBase.java |    2 +-
 .../GridHadoopExternalCommunication.java        |   14 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |    6 +-
 .../v2/GridHadoopV2JobResourceManager.java      |    2 +-
 .../GridHadoopClientProtocolSelfTest.java       |    6 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |    2 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java |    2 +-
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |    2 +-
 .../IgfsHadoopFileSystemAbstractSelfTest.java   |    1 +
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |    2 +-
 .../hadoop/GridHadoopGroupingTest.java          |    4 +-
 .../igfs/IgfsPerformanceBenchmark.java          |    9 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    7 +-
 modules/hibernate/pom.xml                       |    6 -
 .../HibernateReadWriteAccessStrategy.java       |   81 +-
 modules/indexing/pom.xml                        |    6 -
 modules/jta/pom.xml                             |    6 -
 modules/scalar/pom.xml                          |    6 -
 modules/spring/pom.xml                          |    6 -
 modules/visor-console/pom.xml                   |    7 -
 .../commands/alert/VisorAlertCommand.scala      |    8 +-
 .../commands/cache/VisorCacheCommand.scala      |   82 +-
 .../config/VisorConfigurationCommand.scala      |  140 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   64 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |    2 +-
 modules/web/pom.xml                             |    6 -
 modules/winservice/IgniteService.sln            |    2 +-
 .../IgniteService/IgniteService.csproj          |    2 +-
 .../config/benchmark-atomic-win.properties      |   15 +
 .../config/benchmark-atomic.properties          |   15 +
 .../config/benchmark-compute-win.properties     |   15 +
 .../config/benchmark-compute.properties         |   15 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-query-win.properties       |   15 +
 .../yardstick/config/benchmark-query.properties |   15 +
 .../config/benchmark-tx-win.properties          |   15 +
 .../yardstick/config/benchmark-tx.properties    |   15 +
 .../yardstick/config/benchmark-win.properties   |   15 +
 modules/yardstick/config/benchmark.properties   |   15 +
 pom.xml                                         |  150 +-
 106 files changed, 1568 insertions(+), 2614 deletions(-)
----------------------------------------------------------------------



[32/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-239' into sprint-2

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-239' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: 96714719d98de537accd7c7c571cfdbbabfa948d
Parents: 11efb91 08df446
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 17:05:34 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 17:05:34 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |   5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++++----------
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 3 files changed, 267 insertions(+), 331 deletions(-)
----------------------------------------------------------------------



[27/50] incubator-ignite git commit: #ignite-239: review.

Posted by ak...@apache.org.
#ignite-239: 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/08df446b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/08df446b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/08df446b

Branch: refs/heads/ignite-187
Commit: 08df446b39f28b7a4b670650f0e39797c0cfa5c5
Parents: 0b1eccf
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 12:15:13 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 12:15:13 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/IgnitionEx.java  | 10 ++++------
 1 file changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/08df446b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index c2c23f7..cb2efbf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1494,6 +1494,8 @@ public class IgnitionEx {
 
             UUID nodeId = cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID();
 
+            myCfg.setNodeId(nodeId);
+
             IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
 
             assert cfgLog != null;
@@ -1563,10 +1565,8 @@ public class IgnitionEx {
                 }
             }
 
-            if (myCfg.getUserAttributes() == null) {
-                Map<String, ?> emptyAttr = Collections.emptyMap();
-                myCfg.setUserAttributes(emptyAttr);
-            }
+            if (myCfg.getUserAttributes() == null)
+                myCfg.setUserAttributes(Collections.<String, Object>emptyMap());
 
             if (myCfg.getMBeanServer() == null)
                 myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
@@ -1606,8 +1606,6 @@ public class IgnitionEx {
             if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null)
                 myCfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);
 
-            myCfg.setNodeId(nodeId);
-
             IgfsConfiguration[] igfsCfgs = myCfg.getIgfsConfiguration();
 
             if (igfsCfgs != null) {


[26/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-237


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

Branch: refs/heads/ignite-187
Commit: 6c67bdf27f962aee64414c90965e8a86a1cc907d
Parents: 6deba53 c9f46c1
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 11:45:01 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 11:45:01 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/IgniteUtils.java       |  90 ++++++++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 137 +++++++++++--------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 116 ++++++++++++++++
 3 files changed, 275 insertions(+), 68 deletions(-)
----------------------------------------------------------------------



[16/50] incubator-ignite git commit: #ignite-237: remove tests for consistent hash.

Posted by ak...@apache.org.
#ignite-237: remove tests for consistent hash.


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

Branch: refs/heads/ignite-187
Commit: 5692d7e39a4600b17dd9fafb2003a96a2fb51489
Parents: e378264
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 19:44:08 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 19:44:08 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionedAffinitySelfTest.java   | 181 -------------------
 1 file changed, 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5692d7e3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
index 1559a24..069ffb6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedAffinitySelfTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
@@ -110,186 +109,6 @@ public class GridCachePartitionedAffinitySelfTest extends GridCommonAbstractTest
         return aff.mapKeyToPrimaryAndBackups(key);
     }
 
-    /** Test predefined affinity - must be ported to all clients. */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefined() throws IgniteCheckedException {
-        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(new CacheAffinityNodeIdHashResolver());
-
-        List<ClusterNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 4);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 7);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 1);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 7);
-
-        data.put(0, 4);
-        data.put(1, 7);
-        data.put(12, 5);
-        data.put(123, 6);
-        data.put(1234, 4);
-        data.put(12345, 6);
-        data.put(123456, 6);
-        data.put(1234567, 6);
-        data.put(12345678, 0);
-        data.put(123456789, 7);
-        data.put(1234567890, 7);
-        data.put(1234567890L, 7);
-        data.put(12345678901L, 2);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 0);
-        data.put(12345678901234L, 1);
-        data.put(123456789012345L, 6);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 4);
-        data.put(-2345678901234L, 1);
-        data.put(-234567890123L, 5);
-        data.put(-23456789012L, 5);
-        data.put(-2345678901L, 7);
-        data.put(-234567890L, 4);
-        data.put(-234567890, 7);
-        data.put(-23456789, 7);
-        data.put(-2345678, 0);
-        data.put(-234567, 6);
-        data.put(-23456, 6);
-        data.put(-2345, 6);
-        data.put(-234, 7);
-        data.put(-23, 5);
-        data.put(-2, 4);
-
-        data.put(0x80000000, 4);
-        data.put(0x7fffffff, 7);
-        data.put(0x8000000000000000L, 4);
-        data.put(0x7fffffffffffffffL, 4);
-
-        data.put(+1.1, 3);
-        data.put(-10.01, 4);
-        data.put(+100.001, 4);
-        data.put(-1000.0001, 4);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 6);
-        data.put(+4.9E-324, 7);
-        data.put(-4.9E-324, 7);
-    }
-
-    /** Test predefined affinity - must be ported to other clients. */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefinedHashIdResolver() throws IgniteCheckedException {
-        // Use Md5 hasher for this test.
-        CacheRendezvousAffinityFunction aff = new CacheRendezvousAffinityFunction();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(new CacheAffinityNodeHashResolver() {
-            @Override public Object resolve(ClusterNode node) {
-                return null;
-            }
-        });
-
-        List<ClusterNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 3);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 4);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 4);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 2);
-
-        data.put(0, 4);
-        data.put(1, 1);
-        data.put(12, 7);
-        data.put(123, 1);
-        data.put(1234, 6);
-        data.put(12345, 2);
-        data.put(123456, 5);
-        data.put(1234567, 4);
-        data.put(12345678, 6);
-        data.put(123456789, 3);
-        data.put(1234567890, 3);
-        data.put(1234567890L, 3);
-        data.put(12345678901L, 0);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 3);
-        data.put(12345678901234L, 5);
-        data.put(123456789012345L, 5);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 6);
-        data.put(-2345678901234L, 4);
-        data.put(-234567890123L, 3);
-        data.put(-23456789012L, 0);
-        data.put(-2345678901L, 4);
-        data.put(-234567890L, 5);
-        data.put(-234567890, 3);
-        data.put(-23456789, 3);
-        data.put(-2345678, 6);
-        data.put(-234567, 4);
-        data.put(-23456, 5);
-        data.put(-2345, 2);
-        data.put(-234, 7);
-        data.put(-23, 6);
-        data.put(-2, 6);
-
-        data.put(0x80000000, 7);
-        data.put(0x7fffffff, 1);
-        data.put(0x8000000000000000L, 7);
-        data.put(0x7fffffffffffffffL, 7);
-
-        data.put(+1.1, 2);
-        data.put(-10.01, 0);
-        data.put(+100.001, 2);
-        data.put(-1000.0001, 0);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 1);
-        data.put(+4.9E-324, 1);
-        data.put(-4.9E-324, 1);
-    }
-
-    /**
-     * Create node with specified node id and replica count.
-     *
-     * @param nodeId Node id.
-     * @param replicaCnt Node partitioned affinity replica count.
-     * @return New node with specified node id and replica count.
-     */
-    private ClusterNode createNode(String nodeId, int replicaCnt) {
-        GridTestNode node = new GridTestNode(UUID.fromString(nodeId));
-
-        return node;
-    }
-
     /** @throws Exception If failed. */
     public void testAffinity() throws Exception {
         waitTopologyUpdate();


[43/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-334

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-2' into ignite-334


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

Branch: refs/heads/ignite-187
Commit: c577685022f9b59342acd2335efeb9149e81049d
Parents: 4df7c85 b2675bc
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 18:12:06 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 18:12:06 2015 +0300

----------------------------------------------------------------------
 examples/pom-standalone.xml                     |   2 +-
 .../java/org/apache/ignite/IgniteCluster.java   |   8 +-
 .../configuration/IgniteConfiguration.java      |   5 +-
 .../ignite/internal/GridKernalContext.java      |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |  14 +-
 .../apache/ignite/internal/IgniteKernal.java    |  23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++++----------
 .../affinity/GridAffinityAssignmentCache.java   |  13 +-
 .../processors/cache/CacheMetricsImpl.java      |   4 +
 .../processors/cache/GridCacheContext.java      |  19 +
 .../processors/cache/GridCacheMapEntry.java     |   9 +
 .../processors/cache/GridCacheProcessor.java    |  15 +
 .../processors/cache/GridCacheStoreManager.java |  12 +-
 .../GridDistributedCacheAdapter.java            |   2 +
 .../distributed/dht/GridDhtCacheEntry.java      |   3 +
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../processors/cluster/ClusterProcessor.java    |  46 ++
 .../dataload/IgniteDataLoaderImpl.java          |  29 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    | 440 ++++++++++++++
 .../cache/GridCacheAbstractMetricsSelfTest.java |  42 ++
 ...acheAbstractUsersAffinityMapperSelfTest.java | 207 +++++++
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCachePartitionedLocalStoreSelfTest.java |  51 ++
 ...chePartitionedOffHeapLocalStoreSelfTest.java |  56 ++
 .../GridCacheReplicatedLocalStoreSelfTest.java  |  51 ++
 ...heReplicatedUsersAffinityMapperSelfTest.java |  45 ++
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |  51 ++
 .../GridCacheTxUsersAffinityMapperSelfTest.java |  45 ++
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  38 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 modules/scalar/pom.xml                          |   6 +
 pom.xml                                         |   4 +-
 37 files changed, 1503 insertions(+), 407 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5776850/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------


[23/50] incubator-ignite git commit: Merge branch 'ignite-237' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-237

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


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

Branch: refs/heads/ignite-187
Commit: 6531cbd1d7e6fa937f8536e349c8e8dd9631c17c
Parents: b378307 10ac88a
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 21:23:31 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 21:23:31 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |  38 +-
 .../impl/ClientPartitionAffinitySelfTest.java   |  87 +--
 .../ClientAbstractMultiNodeSelfTest.java        |  53 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../java/org/apache/ignite/cache/GridCache.java |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 703 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../CacheRendezvousAffinityFunction.java        |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../processors/cache/GridCacheAttributes.java   |  32 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../top/GridTopologyCommandHandler.java         |   8 -
 .../cache/VisorCacheAffinityConfiguration.java  |   9 +-
 .../config/load/dsi-49-server-production.xml    |   2 +-
 .../GridCacheAffinityBackupsSelfTest.java       |   9 -
 .../ignite/IgniteCacheAffinitySelfTest.java     |  30 +-
 ...AffinityProcessorConsistentHashSelfTest.java |  31 -
 ...idCacheConfigurationConsistencySelfTest.java |  46 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |  16 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   6 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  14 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   7 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   5 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |  32 -
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   | 226 ------
 .../GridCachePartitionedFullApiSelfTest.java    |   6 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |  19 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../capacity/spring-capacity-cache.xml          |   2 +-
 .../loadtests/colocation/spring-colocation.xml  |   2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  11 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 -
 .../hibernate/HibernateL2CacheSelfTest.java     |   8 +-
 64 files changed, 149 insertions(+), 1454 deletions(-)
----------------------------------------------------------------------



[20/50] incubator-ignite git commit: #ignite-239: small refactoring.

Posted by ak...@apache.org.
#ignite-239: small refactoring.


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

Branch: refs/heads/ignite-187
Commit: 55669d46b838a1312e1067c8608243b89462fc54
Parents: ba010da
Author: ivasilinets <va...@gmail.com>
Authored: Sun Mar 1 15:39:37 2015 +0300
Committer: ivasilinets <va...@gmail.com>
Committed: Sun Mar 1 15:39:37 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 172 +++++++++----------
 1 file changed, 86 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/55669d46/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index c7e7cb6..9770205 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1295,29 +1295,14 @@ public class IgnitionEx {
         private void start0(GridStartContext startCtx) throws IgniteCheckedException {
             assert grid == null : "Grid is already started: " + name;
 
-            // Set configuration URL, if any, into system property.
-            if (startCtx.configUrl() != null)
-                System.setProperty(IGNITE_CONFIG_URL, startCtx.configUrl().toString());
-
             IgniteConfiguration cfg = startCtx.config() != null ? startCtx.config() : new IgniteConfiguration();
 
-            // Ensure invariant.
-            // It's a bit dirty - but this is a result of late refactoring
-            // and I don't want to reshuffle a lot of code.
-            assert F.eq(name, cfg.getGridName());
-
-            // Validate segmentation configuration.
-            GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
-
-            // 1. Warn on potential configuration problem: grid is not configured to wait
-            // for correct segment after segmentation happens.
-            if (!F.isEmpty(cfg.getSegmentationResolvers()) && segPlc == RESTART_JVM && !cfg.isWaitForSegmentOnStart()) {
-                U.warn(log, "Found potential configuration problem (forgot to enable waiting for segment" +
-                    "on start?) [segPlc=" + segPlc + ", wait=false]");
-            }
-
             IgniteConfiguration myCfg = initializeConfiguration(cfg);
 
+            // Set configuration URL, if any, into system property.
+            if (startCtx.configUrl() != null)
+                System.setProperty(IGNITE_CONFIG_URL, startCtx.configUrl().toString());
+
             // Ensure that SPIs support multiple grid instances, if required.
             if (!startCtx.single()) {
                 ensureMultiInstanceSupport(myCfg.getDeploymentSpi());
@@ -1331,17 +1316,6 @@ public class IgnitionEx {
                 ensureMultiInstanceSupport(myCfg.getSwapSpaceSpi());
             }
 
-            try {
-                // Use reflection to avoid loading undesired classes.
-                Class helperCls = Class.forName("org.apache.ignite.util.GridConfigurationHelper");
-
-                helperCls.getMethod("overrideConfiguration", IgniteConfiguration.class, Properties.class,
-                    String.class, IgniteLogger.class).invoke(helperCls, myCfg, System.getProperties(), name, log);
-            }
-            catch (Exception ignored) {
-                // No-op.
-            }
-
             execSvc = new IgniteThreadPoolExecutor(
                 "pub-" + cfg.getGridName(),
                 cfg.getPublicThreadPoolSize(),
@@ -1414,6 +1388,17 @@ public class IgnitionEx {
             // Register Ignite MBean for current grid instance.
             registerFactoryMbean(myCfg.getMBeanServer());
 
+            try {
+                // Use reflection to avoid loading undesired classes.
+                Class helperCls = Class.forName("org.apache.ignite.util.GridConfigurationHelper");
+
+                helperCls.getMethod("overrideConfiguration", IgniteConfiguration.class, Properties.class,
+                        String.class, IgniteLogger.class).invoke(helperCls, myCfg, System.getProperties(), name, log);
+            }
+            catch (Exception ignored) {
+                // No-op.
+            }
+
             boolean started = false;
 
             try {
@@ -1489,16 +1474,7 @@ public class IgnitionEx {
          */
         private IgniteConfiguration initializeConfiguration(IgniteConfiguration cfg)
             throws IgniteCheckedException {
-            // Initialize factory's log.
-            UUID nodeId = cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID();
-
-            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
-
-            assert cfgLog != null;
-
-            cfgLog = new GridLoggerProxy(cfgLog, null, name, U.id8(nodeId));
-
-            log = cfgLog.getLogger(G.class);
+            IgniteConfiguration myCfg = new IgniteConfiguration(cfg);
 
             String ggHome = cfg.getIgniteHome();
 
@@ -1511,6 +1487,24 @@ public class IgnitionEx {
 
             U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
 
+            // Ensure invariant.
+            // It's a bit dirty - but this is a result of late refactoring
+            // and I don't want to reshuffle a lot of code.
+            assert F.eq(name, cfg.getGridName());
+
+            UUID nodeId = cfg.getNodeId() != null ? cfg.getNodeId() : UUID.randomUUID();
+
+            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
+
+            assert cfgLog != null;
+
+            cfgLog = new GridLoggerProxy(cfgLog, null, name, U.id8(nodeId));
+
+            // Initialize factory's log.
+            log = cfgLog.getLogger(G.class);
+
+            myCfg.setGridLogger(cfgLog);
+
             // Check Ignite home folder (after log is available).
             if (ggHome != null) {
                 File ggHomeFile = new File(ggHome);
@@ -1519,11 +1513,23 @@ public class IgnitionEx {
                     throw new IgniteCheckedException("Invalid Ignite installation home folder: " + ggHome);
             }
 
-            IgniteConfiguration myCfg = new IgniteConfiguration(cfg);
-
             myCfg.setIgniteHome(ggHome);
 
-            myCfg.setGridLogger(cfgLog);
+            // Validate segmentation configuration.
+            GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
+
+            // 1. Warn on potential configuration problem: grid is not configured to wait
+            // for correct segment after segmentation happens.
+            if (!F.isEmpty(cfg.getSegmentationResolvers()) && segPlc == RESTART_JVM && !cfg.isWaitForSegmentOnStart()) {
+                U.warn(log, "Found potential configuration problem (forgot to enable waiting for segment" +
+                        "on start?) [segPlc=" + segPlc + ", wait=false]");
+            }
+
+            myCfg.setTransactionConfiguration(myCfg.getTransactionConfiguration() != null ?
+                    new TransactionConfiguration(myCfg.getTransactionConfiguration()) : null);
+
+            myCfg.setConnectorConfiguration(myCfg.getConnectorConfiguration() != null ?
+                    new ConnectorConfiguration(myCfg.getConnectorConfiguration()) : null);
 
             // Local host.
             String locHost = IgniteSystemProperties.getString(IGNITE_LOCAL_HOST);
@@ -1534,6 +1540,37 @@ public class IgnitionEx {
             if (daemon)
                 myCfg.setDaemon(true);
 
+            // Check for deployment mode override.
+            String depModeName = IgniteSystemProperties.getString(IGNITE_DEP_MODE_OVERRIDE);
+
+            if (!F.isEmpty(depModeName)) {
+                if (!F.isEmpty(myCfg.getCacheConfiguration())) {
+                    U.quietAndInfo(log, "Skipping deployment mode override for caches (custom closure " +
+                            "execution may not work for console Visor)");
+                }
+                else {
+                    try {
+                        DeploymentMode depMode = DeploymentMode.valueOf(depModeName);
+
+                        if (myCfg.getDeploymentMode() != depMode)
+                            myCfg.setDeploymentMode(depMode);
+                    }
+                    catch (IllegalArgumentException e) {
+                        throw new IgniteCheckedException("Failed to override deployment mode using system property " +
+                                "(are there any misspellings?)" +
+                                "[name=" + IGNITE_DEP_MODE_OVERRIDE + ", value=" + depModeName + ']', e);
+                    }
+                }
+            }
+
+            if (myCfg.getUserAttributes() == null) {
+                Map<String, ?> emptyAttr = Collections.emptyMap();
+                myCfg.setUserAttributes(emptyAttr);
+            }
+
+            if (myCfg.getMBeanServer() == null)
+                myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
+
             Marshaller marsh = myCfg.getMarshaller();
 
             if (marsh == null) {
@@ -1564,33 +1601,12 @@ public class IgnitionEx {
                     "Using GridOptimizedMarshaller on untested JVM.");
             }
 
-            // Check for deployment mode override.
-            String depModeName = IgniteSystemProperties.getString(IGNITE_DEP_MODE_OVERRIDE);
-
-            if (!F.isEmpty(depModeName)) {
-                if (!F.isEmpty(myCfg.getCacheConfiguration())) {
-                    U.quietAndInfo(log, "Skipping deployment mode override for caches (custom closure " +
-                        "execution may not work for console Visor)");
-                }
-                else {
-                    try {
-                        DeploymentMode depMode = DeploymentMode.valueOf(depModeName);
-
-                        if (myCfg.getDeploymentMode() != depMode)
-                            myCfg.setDeploymentMode(depMode);
-                    }
-                    catch (IllegalArgumentException e) {
-                        throw new IgniteCheckedException("Failed to override deployment mode using system property " +
-                            "(are there any misspellings?)" +
-                            "[name=" + IGNITE_DEP_MODE_OVERRIDE + ", value=" + depModeName + ']', e);
-                    }
-                }
-            }
-
             myCfg.setMarshaller(marsh);
 
-            myCfg.setConnectorConfiguration(myCfg.getConnectorConfiguration() != null ?
-                new ConnectorConfiguration(myCfg.getConnectorConfiguration()) : null);
+            if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null)
+                myCfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);
+
+            myCfg.setNodeId(nodeId);
 
             IgfsConfiguration[] igfsCfgs = myCfg.getIgfsConfiguration();
 
@@ -1603,14 +1619,6 @@ public class IgnitionEx {
                 myCfg.setIgfsConfiguration(clone);
             }
 
-            if (myCfg.getMBeanServer() == null)
-                myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
-
-            myCfg.setNodeId(nodeId);
-
-            if (myCfg.getPeerClassLoadingLocalClassPathExclude() == null)
-                myCfg.setPeerClassLoadingLocalClassPathExclude(EMPTY_STR_ARR);
-
             StreamerConfiguration[] streamerCfgs = myCfg.getStreamerConfiguration();
 
             if (streamerCfgs != null) {
@@ -1622,18 +1630,10 @@ public class IgnitionEx {
                 myCfg.setStreamerConfiguration(clone);
             }
 
-            myCfg.setTransactionConfiguration(myCfg.getTransactionConfiguration() != null ?
-                new TransactionConfiguration(myCfg.getTransactionConfiguration()) : null);
-
-            if (myCfg.getUserAttributes() == null) {
-                Map<String, ?> emptyAttr = Collections.emptyMap();
-                myCfg.setUserAttributes(emptyAttr);
-            }
+            initializeDefaultSpi(myCfg);
 
             initializeDefaultCacheConfiguration(myCfg);
 
-            initializeDefaultSpi(myCfg);
-
             return myCfg;
         }
 
@@ -1722,7 +1722,7 @@ public class IgnitionEx {
         }
 
         /**
-         * Initialize default values for spi.
+         * Initialize default SPI implementations.
          *
          * @param cfg Ignite configuration.
          */


[14/50] incubator-ignite git commit: #ignite-237: return to Visor excludeNeighbors attribute.

Posted by ak...@apache.org.
#ignite-237: return to Visor excludeNeighbors attribute.


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

Branch: refs/heads/ignite-187
Commit: 414410bea39a7d8bdc46b1b63c5f506ec96d653e
Parents: 936b879
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Feb 27 18:34:08 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Feb 27 18:34:08 2015 +0300

----------------------------------------------------------------------
 .../visor/cache/VisorCacheAffinityConfiguration.java        | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/414410be/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index c43dc1c..47a2cf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -58,15 +59,19 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     public static VisorCacheAffinityConfiguration from(CacheConfiguration ccfg) {
         CacheAffinityFunction aff = ccfg.getAffinity();
 
-        Integer dfltReplicas = null;
         Boolean excludeNeighbors = null;
 
+        if (aff instanceof CacheRendezvousAffinityFunction) {
+            CacheRendezvousAffinityFunction hashAffFunc = (CacheRendezvousAffinityFunction)aff;
+
+            excludeNeighbors = hashAffFunc.isExcludeNeighbors();
+        }
+
         VisorCacheAffinityConfiguration cfg = new VisorCacheAffinityConfiguration();
 
         cfg.function(compactClass(aff));
         cfg.mapper(compactClass(ccfg.getAffinityMapper()));
         cfg.partitionedBackups(ccfg.getBackups());
-        cfg.defaultReplicas(dfltReplicas);
         cfg.excludeNeighbors(excludeNeighbors);
 
         return cfg;


[48/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-237' into sprint-2

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-237' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: d33abdba733987609f592b046d87209e976ba616
Parents: e5e2a8e f144a89
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Mar 3 11:49:54 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Mar 3 11:49:54 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |  37 +-
 .../impl/ClientPartitionAffinitySelfTest.java   | 406 -----------
 .../ClientAbstractMultiNodeSelfTest.java        |  54 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |   4 -
 .../client/suite/IgniteClientTestSuite.java     |   1 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../java/org/apache/ignite/cache/GridCache.java |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 702 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../CacheRendezvousAffinityFunction.java        |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../ignite/internal/client/GridClientNode.java  |   8 -
 .../client/GridClientPartitionAffinity.java     |   2 +-
 .../client/impl/GridClientNodeImpl.java         |  22 -
 .../connection/GridClientNioTcpConnection.java  |   3 +-
 .../processors/cache/GridCacheAttributes.java   |  32 +-
 .../processors/cache/GridCacheProcessor.java    |  29 +-
 .../rest/client/message/GridClientNodeBean.java |  23 -
 .../top/GridTopologyCommandHandler.java         |   8 -
 .../cache/VisorCacheAffinityConfiguration.java  |   9 +-
 .../config/load/dsi-49-server-production.xml    |   2 +-
 .../GridCacheAffinityBackupsSelfTest.java       |   9 -
 .../ignite/IgniteCacheAffinitySelfTest.java     |  30 +-
 ...AffinityProcessorConsistentHashSelfTest.java |  31 -
 ...idCacheConfigurationConsistencySelfTest.java |  46 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../cache/GridCacheOffHeapSelfTest.java         |  12 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |  16 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |  13 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  13 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   7 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   5 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |  32 -
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   | 226 ------
 .../GridCachePartitionedFullApiSelfTest.java    |  11 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../capacity/spring-capacity-cache.xml          |   2 +-
 .../loadtests/colocation/spring-colocation.xml  |   2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  11 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 -
 .../hibernate/HibernateL2CacheSelfTest.java     |   8 +-
 .../processors/cache/GridCacheSwapSelfTest.java |  12 +-
 73 files changed, 160 insertions(+), 1852 deletions(-)
----------------------------------------------------------------------



[47/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-334' into sprint-2

Posted by ak...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-334' into sprint-2


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

Branch: refs/heads/ignite-187
Commit: e5e2a8e1f1881abf80420d4bf1b64d2775f6cf29
Parents: b81c6a2 c577685
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Mar 3 11:45:03 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Mar 3 11:45:03 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/configuration/IgniteConfiguration.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[50/50] incubator-ignite git commit: Merge branches 'ignite-187' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-187

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


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

Branch: refs/heads/ignite-187
Commit: ba3fcbe1f5f1334765abb0958c47982eb2c6fc08
Parents: 4ba2862 2a68e64
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Mar 3 17:40:27 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Mar 3 17:40:27 2015 +0700

----------------------------------------------------------------------
 examples/pom-standalone.xml                     |   2 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |  37 +-
 .../impl/ClientPartitionAffinitySelfTest.java   | 406 -----------
 .../ClientAbstractMultiNodeSelfTest.java        |  54 +-
 .../integration/ClientAbstractSelfTest.java     |   8 -
 .../ClientTcpUnreachableMultiNodeSelfTest.java  |   4 -
 .../client/suite/IgniteClientTestSuite.java     |   1 -
 .../src/test/resources/spring-server-node.xml   |   2 +-
 .../test/resources/spring-server-ssl-node.xml   |   2 +-
 .../java/org/apache/ignite/IgniteCluster.java   |   8 +-
 .../ignite/cache/CacheTypeFieldMetadata.java    |   2 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |  18 +-
 .../java/org/apache/ignite/cache/GridCache.java |   2 +-
 .../CacheConsistentHashAffinityFunction.java    | 702 -------------------
 .../cache/affinity/consistenthash/package.html  |  24 -
 .../CacheRendezvousAffinityFunction.java        |   2 +-
 .../configuration/CacheConfiguration.java       |   2 +-
 .../configuration/IgniteConfiguration.java      |  11 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 592 +++++++---------
 .../ignite/internal/client/GridClientNode.java  |   8 -
 .../client/GridClientPartitionAffinity.java     |   2 +-
 .../client/impl/GridClientNodeImpl.java         |  22 -
 .../connection/GridClientNioTcpConnection.java  |   3 +-
 .../affinity/GridAffinityAssignmentCache.java   |  13 +-
 .../processors/cache/CacheMetricsImpl.java      |   4 +
 .../processors/cache/GridCacheAttributes.java   |  32 +-
 .../processors/cache/GridCacheContext.java      |  19 +
 .../processors/cache/GridCacheMapEntry.java     |   9 +
 .../processors/cache/GridCacheProcessor.java    |  44 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   3 +
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../rest/client/message/GridClientNodeBean.java |  23 -
 .../top/GridTopologyCommandHandler.java         |   8 -
 .../cache/VisorCacheAffinityConfiguration.java  |  19 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../config/load/dsi-49-server-production.xml    |   2 +-
 .../GridCacheAffinityBackupsSelfTest.java       |   9 -
 .../ignite/IgniteCacheAffinitySelfTest.java     |  30 +-
 ...AffinityProcessorConsistentHashSelfTest.java |  31 -
 .../GridCacheAbstractLocalStoreSelfTest.java    | 440 ++++++++++++
 .../cache/GridCacheAbstractMetricsSelfTest.java |  42 ++
 ...acheAbstractUsersAffinityMapperSelfTest.java | 207 ++++++
 ...dCacheAtomicUsersAffinityMapperSelfTest.java |  45 ++
 ...idCacheConfigurationConsistencySelfTest.java |  46 +-
 ...ridCacheConfigurationValidationSelfTest.java |   8 +-
 .../cache/GridCacheOffHeapSelfTest.java         |  12 +-
 .../GridCachePartitionedAffinitySpreadTest.java |  16 +-
 .../GridCachePartitionedLocalStoreSelfTest.java |  51 ++
 ...chePartitionedOffHeapLocalStoreSelfTest.java |  56 ++
 .../GridCacheReplicatedLocalStoreSelfTest.java  |  51 ++
 ...heReplicatedUsersAffinityMapperSelfTest.java |  45 ++
 ...ridCacheTxPartitionedLocalStoreSelfTest.java |  51 ++
 .../GridCacheTxUsersAffinityMapperSelfTest.java |  45 ++
 .../IgniteClientAffinityAssignmentSelfTest.java |  16 +-
 .../GridCacheQueueCleanupSelfTest.java          |   1 -
 .../GridCacheClientModesAbstractSelfTest.java   |   4 +-
 ...dCachePartitionedAffinityFilterSelfTest.java |   6 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |   4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |  13 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |  13 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |   4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |   4 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |   4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   4 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   4 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |   4 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |   6 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   7 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |   5 +-
 ...ffinityFunctionExcludeNeighborsSelfTest.java |  32 -
 ...AffinityExcludeNeighborsPerformanceTest.java |   4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   4 +-
 .../GridCachePartitionedAffinitySelfTest.java   | 226 ------
 .../GridCachePartitionedFullApiSelfTest.java    |  11 +-
 .../GridCachePartitionedNodeRestartTest.java    |   4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |   4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   4 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../capacity/spring-capacity-cache.xml          |   2 +-
 .../loadtests/colocation/spring-colocation.xml  |   2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |   4 +-
 ...apPartitionedMapPerformanceAbstractTest.java |   4 +-
 ...GridUnsafePartitionedMapPerformanceTest.java |   4 +-
 .../ignite/testframework/GridTestUtils.java     |  11 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |   5 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   8 +-
 .../processors/cache/GridCacheSwapSelfTest.java |  12 +-
 modules/scalar/pom.xml                          |   6 +
 .../ignite/schema/generator/PojoGenerator.java  |   3 +-
 .../schema/generator/SnippetGenerator.java      | 138 ++++
 .../apache/ignite/schema/ui/SchemaLoadApp.java  |   2 +
 .../commands/cache/VisorCacheCommand.scala      |  18 +-
 .../commands/cache/VisorCacheScanCommand.scala  |   7 +-
 .../config/VisorConfigurationCommand.scala      |   5 +-
 .../visor/commands/node/VisorNodeCommand.scala  |   6 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  27 +-
 pom.xml                                         |   4 +-
 110 files changed, 1741 insertions(+), 2249 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ba3fcbe1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------


[30/50] incubator-ignite git commit: #ignite-237: remove test for consistent hash replicaCount.

Posted by ak...@apache.org.
#ignite-237: remove test for consistent hash replicaCount.


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

Branch: refs/heads/ignite-187
Commit: 5a76d70f99c276f95cfdb4ae97c09341fefc468a
Parents: 5f620d6
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Mar 2 12:49:56 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Mar 2 12:49:56 2015 +0300

----------------------------------------------------------------------
 .../impl/ClientPartitionAffinitySelfTest.java   | 323 -------------------
 .../client/suite/IgniteClientTestSuite.java     |   1 -
 2 files changed, 324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5a76d70f/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
deleted file mode 100644
index 6af1c4c..0000000
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPartitionAffinitySelfTest.java
+++ /dev/null
@@ -1,323 +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.client.impl;
-
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.client.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.util.*;
-
-/**
- * Client's partitioned affinity tests.
- */
-public class ClientPartitionAffinitySelfTest extends GridCommonAbstractTest {
-    /** Hash ID resolver. */
-    private static final GridClientPartitionAffinity.HashIdResolver HASH_ID_RSLVR =
-        new GridClientPartitionAffinity.HashIdResolver() {
-            @Override public Object getHashId(GridClientNode node) {
-                return node.nodeId();
-            }
-        };
-
-    /**
-     * Test predefined affinity - must be ported to other clients.
-     */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefined() throws Exception {
-        // Use Md5 hasher for this test.
-        GridClientPartitionAffinity aff = new GridClientPartitionAffinity();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(HASH_ID_RSLVR);
-
-        List<GridClientNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 4);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 7);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 1);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 7);
-
-        data.put(0, 4);
-        data.put(1, 7);
-        data.put(12, 5);
-        data.put(123, 6);
-        data.put(1234, 4);
-        data.put(12345, 6);
-        data.put(123456, 6);
-        data.put(1234567, 6);
-        data.put(12345678, 0);
-        data.put(123456789, 7);
-        data.put(1234567890, 7);
-        data.put(1234567890L, 7);
-        data.put(12345678901L, 2);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 0);
-        data.put(12345678901234L, 1);
-        data.put(123456789012345L, 6);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 4);
-        data.put(-2345678901234L, 1);
-        data.put(-234567890123L, 5);
-        data.put(-23456789012L, 5);
-        data.put(-2345678901L, 7);
-        data.put(-234567890L, 4);
-        data.put(-234567890, 7);
-        data.put(-23456789, 7);
-        data.put(-2345678, 0);
-        data.put(-234567, 6);
-        data.put(-23456, 6);
-        data.put(-2345, 6);
-        data.put(-234, 7);
-        data.put(-23, 5);
-        data.put(-2, 4);
-
-        data.put(0x80000000, 4);
-        data.put(0x7fffffff, 7);
-        data.put(0x8000000000000000L, 4);
-        data.put(0x7fffffffffffffffL, 4);
-
-        data.put(+1.1, 3);
-        data.put(-10.01, 4);
-        data.put(+100.001, 4);
-        data.put(-1000.0001, 4);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 6);
-        data.put(+4.9E-324, 7);
-        data.put(-4.9E-324, 7);
-
-        boolean ok = true;
-
-        for (Map.Entry<Object, Integer> entry : data.entrySet()) {
-            UUID exp = nodes.get(entry.getValue()).nodeId();
-            UUID act = aff.node(entry.getKey(), nodes).nodeId();
-
-            if (exp.equals(act))
-                continue;
-
-            ok = false;
-
-            info("Failed to validate affinity for key '" + entry.getKey() + "' [expected=" + exp +
-                ", actual=" + act + ".");
-        }
-
-        if (ok)
-            return;
-
-        fail("Client partitioned affinity validation fails.");
-    }
-
-    /**
-     * Test predefined affinity - must be ported to other clients.
-     */
-    @SuppressWarnings("UnaryPlus")
-    public void testPredefinedHashIdResolver() throws Exception {
-        // Use Md5 hasher for this test.
-        GridClientPartitionAffinity aff = new GridClientPartitionAffinity();
-
-        getTestResources().inject(aff);
-
-        aff.setHashIdResolver(new GridClientPartitionAffinity.HashIdResolver() {
-            @Override public Object getHashId(GridClientNode node) {
-                return 1;
-            }
-        });
-
-        List<GridClientNode> nodes = new ArrayList<>();
-
-        nodes.add(createNode("000ea4cd-f449-4dcb-869a-5317c63bd619", 50));
-        nodes.add(createNode("010ea4cd-f449-4dcb-869a-5317c63bd62a", 60));
-        nodes.add(createNode("0209ec54-ff53-4fdb-8239-5a3ac1fb31bd", 70));
-        nodes.add(createNode("0309ec54-ff53-4fdb-8239-5a3ac1fb31ef", 80));
-        nodes.add(createNode("040c9b94-02ae-45a6-9d5c-a066dbdf2636", 90));
-        nodes.add(createNode("050c9b94-02ae-45a6-9d5c-a066dbdf2747", 100));
-        nodes.add(createNode("0601f916-4357-4cfe-a7df-49d4721690bf", 110));
-        nodes.add(createNode("0702f916-4357-4cfe-a7df-49d4721691c0", 120));
-
-        Map<Object, Integer> data = new LinkedHashMap<>();
-
-        data.put("", 4);
-        data.put("asdf", 3);
-        data.put("224ea4cd-f449-4dcb-869a-5317c63bd619", 5);
-        data.put("fdc9ec54-ff53-4fdb-8239-5a3ac1fb31bd", 2);
-        data.put("0f9c9b94-02ae-45a6-9d5c-a066dbdf2636", 2);
-        data.put("d8f1f916-4357-4cfe-a7df-49d4721690bf", 4);
-        data.put("c77ffeae-78a1-4ee6-a0fd-8d197a794412", 3);
-        data.put("35de9f21-3c9b-4f4a-a7d5-3e2c6cb01564", 4);
-        data.put("d67eb652-4e76-47fb-ad4e-cd902d9b868a", 2);
-
-        data.put(0, 4);
-        data.put(1, 1);
-        data.put(12, 7);
-        data.put(123, 1);
-        data.put(1234, 6);
-        data.put(12345, 2);
-        data.put(123456, 5);
-        data.put(1234567, 4);
-        data.put(12345678, 6);
-        data.put(123456789, 3);
-        data.put(1234567890, 3);
-        data.put(1234567890L, 3);
-        data.put(12345678901L, 0);
-        data.put(123456789012L, 1);
-        data.put(1234567890123L, 3);
-        data.put(12345678901234L, 5);
-        data.put(123456789012345L, 5);
-        data.put(1234567890123456L, 7);
-        data.put(-23456789012345L, 6);
-        data.put(-2345678901234L, 4);
-        data.put(-234567890123L, 3);
-        data.put(-23456789012L, 0);
-        data.put(-2345678901L, 4);
-        data.put(-234567890L, 5);
-        data.put(-234567890, 3);
-        data.put(-23456789, 3);
-        data.put(-2345678, 6);
-        data.put(-234567, 4);
-        data.put(-23456, 5);
-        data.put(-2345, 2);
-        data.put(-234, 7);
-        data.put(-23, 6);
-        data.put(-2, 6);
-
-        data.put(0x80000000, 7);
-        data.put(0x7fffffff, 1);
-        data.put(0x8000000000000000L, 7);
-        data.put(0x7fffffffffffffffL, 7);
-
-        data.put(+1.1, 2);
-        data.put(-10.01, 0);
-        data.put(+100.001, 2);
-        data.put(-1000.0001, 0);
-        data.put(+1.7976931348623157E+308, 6);
-        data.put(-1.7976931348623157E+308, 1);
-        data.put(+4.9E-324, 1);
-        data.put(-4.9E-324, 1);
-
-        boolean ok = true;
-
-        for (Map.Entry<Object, Integer> entry : data.entrySet()) {
-            UUID exp = nodes.get(entry.getValue()).nodeId();
-            UUID act = aff.node(entry.getKey(), nodes).nodeId();
-
-            if (exp.equals(act))
-                continue;
-
-            ok = false;
-
-            info("Failed to validate affinity for key '" + entry.getKey() + "' [expected=" + exp +
-                ", actual=" + act + ".");
-        }
-
-        if (ok)
-            return;
-
-        fail("Client partitioned affinity validation fails.");
-    }
-
-    /**
-     * Create node with specified node id and replica count.
-     *
-     * @param nodeId Node id.
-     * @param replicaCnt Node partitioned affinity replica count.
-     * @return New node with specified node id and replica count.
-     */
-    private GridClientNode createNode(String nodeId, int replicaCnt) {
-        return GridClientNodeImpl.builder()
-            .nodeId(UUID.fromString(nodeId))
-            .build();
-    }
-
-    /**
-     * Add {@code cnt} nodes into emulated topology.
-     *
-     * @param cnt Number of nodes to add into emulated topology.
-     * @param nodes Client topology.
-     * @param srvNodes Server topology.
-     */
-    private void addNodes(int cnt, Collection<GridClientNode> nodes, Collection<ClusterNode> srvNodes) {
-        while (cnt-- > 0) {
-            UUID nodeId = UUID.randomUUID();
-            int replicaCnt = (int)Math.round(Math.random() * 500) + 1;
-
-            nodes.add(GridClientNodeImpl.builder()
-                .nodeId(nodeId)
-                .build());
-
-            ClusterNode srvNode = new TestRichNode(nodeId);
-
-            srvNodes.add(srvNode);
-        }
-    }
-
-    /**
-     * Rich node stub to use in emulated server topology.
-     */
-    private static class TestRichNode extends GridTestNode {
-        /**
-         * Node id.
-         */
-        private final UUID nodeId;
-
-
-        /**
-         * Externalizable class requires public no-arg constructor.
-         */
-        @SuppressWarnings("UnusedDeclaration")
-        public TestRichNode() {
-            this(UUID.randomUUID());
-        }
-
-        /**
-         * Constructs rich node stub to use in emulated server topology.
-         *
-         * @param nodeId Node id.
-         */
-        private TestRichNode(UUID nodeId) {
-            this.nodeId = nodeId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public UUID id() {
-            return nodeId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T> T attribute(String name) {
-            return super.attribute(name);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5a76d70f/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
index d89a960..4785248 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
@@ -95,7 +95,6 @@ public class IgniteClientTestSuite extends TestSuite {
         suite.addTest(new TestSuite(ClientDefaultCacheSelfTest.class));
 
         suite.addTestSuite(ClientFutureAdapterSelfTest.class);
-        suite.addTestSuite(ClientPartitionAffinitySelfTest.class);
         suite.addTestSuite(ClientPropertiesConfigurationSelfTest.class);
         suite.addTestSuite(ClientConsistentHashSelfTest.class);
         suite.addTestSuite(ClientJavaHasherSelfTest.class);