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

[01/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-37 0a945ade0 -> 3faca8244 (forced update)


Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

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

Branch: refs/heads/ignite-37
Commit: 9e4bc1044c7b6b3cc2bcbad4cae9bbebd33bcc14
Parents: d24064a 2dfc187
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 24 19:14:24 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 24 19:14:24 2015 +0300

----------------------------------------------------------------------
 .../aop/aspectj/GridifyAspectJAspect.java       |    2 +-
 .../aspectj/GridifySetToSetAspectJAspect.java   |    2 +-
 .../aspectj/GridifySetToValueAspectJAspect.java |    2 +-
 .../aop/spring/GridifySetToSetSpringAspect.java |    2 +-
 .../spring/GridifySetToValueSpringAspect.java   |    2 +-
 .../gridify/aop/spring/GridifySpringAspect.java |    2 +-
 .../cache/CacheServerNotFoundException.java     |   12 +-
 .../apache/ignite/cache/CachingProvider.java    |    3 +
 .../ignite/events/CacheQueryExecutedEvent.java  |    3 +-
 .../ignite/events/CacheQueryReadEvent.java      |    3 +-
 .../apache/ignite/internal/IgniteKernal.java    |   15 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    6 +
 .../ClusterTopologyServerNotFoundException.java |   12 +-
 .../deployment/GridDeploymentClassLoader.java   |    2 +-
 .../discovery/GridDiscoveryManager.java         |    3 +
 .../eventstorage/GridEventStorageManager.java   |    6 +
 .../affinity/GridAffinityAssignmentCache.java   |    4 +-
 .../cache/CacheStoreBalancingWrapper.java       |    6 +
 .../processors/cache/GridCacheAdapter.java      |    9 +-
 .../processors/cache/GridCacheEntryEx.java      |    4 +
 .../processors/cache/GridCacheIoManager.java    |    6 +
 .../processors/cache/GridCacheMapEntry.java     |   20 +-
 .../GridCachePartitionExchangeManager.java      |    2 +-
 .../processors/cache/GridCacheUtils.java        |   53 +-
 .../GridDistributedCacheAdapter.java            |   20 +-
 .../distributed/GridDistributedLockRequest.java |   99 +-
 .../GridDistributedTxRemoteAdapter.java         |    3 +
 .../distributed/dht/GridDhtCacheAdapter.java    |    4 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    9 +-
 .../distributed/dht/GridDhtLockFuture.java      |   20 +-
 .../distributed/dht/GridDhtLockRequest.java     |    7 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   21 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |    3 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   26 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    6 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |    7 +-
 .../dht/GridPartitionedGetFuture.java           |    7 +
 .../dht/atomic/GridDhtAtomicCache.java          |   41 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   38 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   49 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   56 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   18 +-
 .../GridDhtPartitionsExchangeFuture.java        |    3 +
 .../distributed/near/GridNearAtomicCache.java   |    5 +-
 .../distributed/near/GridNearCacheAdapter.java  |   10 +-
 .../distributed/near/GridNearCacheEntry.java    |   10 +-
 .../distributed/near/GridNearGetFuture.java     |   17 +-
 .../distributed/near/GridNearLockFuture.java    |   15 +-
 .../distributed/near/GridNearLockRequest.java   |    7 +-
 .../near/GridNearTransactionalCache.java        |   43 +-
 .../near/GridNearTxFinishFuture.java            |    3 +
 .../cache/distributed/near/GridNearTxLocal.java |    7 +-
 .../distributed/near/GridNearTxRemote.java      |    7 +-
 .../local/atomic/GridLocalAtomicCache.java      |   59 +-
 .../query/GridCacheDistributedQueryManager.java |    3 +
 .../cache/query/GridCacheLocalQueryFuture.java  |    3 +
 .../query/GridCacheQueryFutureAdapter.java      |    3 +
 .../cache/query/GridCacheQueryManager.java      |   20 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |    3 +
 .../cache/transactions/IgniteTxAdapter.java     |    7 +-
 .../cache/transactions/IgniteTxEntry.java       |   74 +-
 .../cache/transactions/IgniteTxHandler.java     |   27 +-
 .../transactions/IgniteTxLocalAdapter.java      |  106 +-
 .../cache/transactions/IgniteTxLocalEx.java     |    4 +-
 .../closure/GridClosureProcessor.java           |   16 +-
 .../datastreamer/DataStreamProcessor.java       |   28 +-
 .../datastreamer/DataStreamerImpl.java          |    3 +-
 .../datastreamer/DataStreamerRequest.java       |   38 +-
 .../processors/igfs/IgfsFileWorkerBatch.java    |    3 +
 .../internal/processors/igfs/IgfsThread.java    |    8 +-
 .../internal/processors/job/GridJobWorker.java  |    9 +
 .../portable/GridPortableInputStream.java       |   26 -
 .../processors/query/GridQueryProcessor.java    |    3 +
 .../processors/rest/GridRestProcessor.java      |    3 +
 .../service/GridServiceProcessor.java           |    9 +
 .../processors/task/GridTaskWorker.java         |   12 +
 .../timeout/GridTimeoutProcessor.java           |    3 +
 .../ignite/internal/util/IgniteUtils.java       |   22 +-
 .../util/ipc/loopback/IpcServerTcpEndpoint.java |    2 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    2 +-
 .../ignite/internal/util/lang/GridFunc.java     |   12 +-
 .../ignite/internal/util/nio/GridNioServer.java |    6 +
 .../apache/ignite/internal/util/typedef/X.java  |    2 +-
 .../ignite/internal/util/worker/GridWorker.java |    3 +
 .../visor/misc/VisorResolveHostNameTask.java    |    2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    6 +-
 .../ignite/messaging/MessagingListenActor.java  |    3 +
 .../org/apache/ignite/spi/IgniteSpiThread.java  |    3 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    3 +
 .../startup/cmdline/CommandLineStartup.java     |    5 +-
 .../startup/cmdline/CommandLineTransformer.java |    3 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |  514 ++++++-
 .../processors/cache/GridCacheTestEntryEx.java  |    2 +
 .../cache/IgniteExcangeFutureHistoryTest.java   |   74 -
 .../cache/IgniteExchangeFutureHistoryTest.java  |   77 ++
 .../CacheNoValueClassOnServerNodeTest.java      |  129 ++
 .../DataStreamerMultiThreadedSelfTest.java      |  101 ++
 .../ignite/testsuites/IgniteCacheTestSuite.java |    7 +-
 .../CacheNoValueClassOnServerTestClient.java    |   88 ++
 .../apache/ignite/tests/p2p/cache/Person.java   |   42 +
 .../CacheConfigurationP2PTestClient.java        |    1 -
 .../processors/hadoop/HadoopDefaultJobInfo.java |    3 +
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    3 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    5 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |    3 +
 .../external/HadoopExternalTaskExecutor.java    |    3 +
 .../processors/hadoop/v2/HadoopV2Job.java       |   11 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |    9 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |    3 +
 .../GridCacheAbstractFieldsQuerySelfTest.java   | 1284 ------------------
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  235 +++-
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |   34 -
 ...GridCachePartitionedFieldsQuerySelfTest.java |  115 --
 ...rtitionedFieldsQueryP2PDisabledSelfTest.java |   34 -
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |   34 +
 ...eReplicatedFieldsQueryP2PEnableSelfTest.java |   34 -
 .../GridCacheReplicatedFieldsQuerySelfTest.java |  157 ---
 ...eplicatedFieldsQueryP2PDisabledSelfTest.java |   34 -
 ...ReplicatedFieldsQueryP2PEnabledSelfTest.java |   34 +
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../http/jetty/GridJettyRestHandler.java        |    3 +
 .../ignite/schema/ui/SchemaImportApp.java       |    8 +-
 .../uri/GridUriDeploymentClassLoader.java       |    4 +-
 123 files changed, 2234 insertions(+), 2141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e4bc104/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index affdaf0,b8668e6..1fe1f50
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@@ -288,7 -282,16 +288,10 @@@ public class GridCacheIoManager extend
              }
          }
          catch (Throwable e) {
 -            if (X.hasCause(e, ClassNotFoundException.class))
 -                U.error(log, "Failed to process message (note that distributed services " +
 -                    "do not support peer class loading, if you deploy distributed service " +
 -                    "you should have all required classes in CLASSPATH on all nodes in topology) " +
 -                    "[senderId=" + nodeId + ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
 -            else
 -                U.error(log, "Failed to process message [senderId=" + nodeId + ']', e);
 +            U.error(log, "Failed to process message [senderId=" + nodeId + ", messageType=" + cacheMsg.getClass() + ']', e);
+ 
+             if (e instanceof Error)
+                 throw (Error)e;
          }
          finally {
              if (depEnabled)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9e4bc104/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------


[45/50] incubator-ignite git commit: IGNITE-856 - gg.client -> ignite.client

Posted by sb...@apache.org.
IGNITE-856 - gg.client -> ignite.client


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

Branch: refs/heads/ignite-37
Commit: 54f949220433d45b543de83896ed3e357a68e04f
Parents: c4bc929
Author: Ignite Teamcity <ig...@apache.org>
Authored: Mon May 4 17:27:14 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 4 17:27:14 2015 -0700

----------------------------------------------------------------------
 .../config/grid-client-config.properties        | 50 ++++++------
 .../ClientPropertiesConfigurationSelfTest.java  | 12 +--
 .../org/apache/ignite/IgniteJdbcDriver.java     | 81 ++++++++++----------
 .../client/GridClientConfiguration.java         |  2 +-
 4 files changed, 71 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/clients/config/grid-client-config.properties
----------------------------------------------------------------------
diff --git a/modules/clients/config/grid-client-config.properties b/modules/clients/config/grid-client-config.properties
index b6bce8d..d25352b 100644
--- a/modules/clients/config/grid-client-config.properties
+++ b/modules/clients/config/grid-client-config.properties
@@ -16,89 +16,89 @@
 #
 
 # Required. Comma-separated list of servers to connect to in format "host:port".
-gg.client.servers=localhost:11211
+ignite.client.servers=localhost:11211
 
 # Optional. Default client load balancer. Default value is "".
 # Balancer may be "random", "roundrobin" or full class name for "your.balancer.ImplementationClass"
-#gg.client.balancer=
+#ignite.client.balancer=
 
 # Optional. Socket connect timeout (ms). Default value is "0".
-#gg.client.connectTimeout=10000
+#ignite.client.connectTimeout=10000
 
 # Optional. Credentials if grid requires authentication. Default value is "".
-#gg.client.credentials=
+#ignite.client.credentials=
 
 # Optional. Flag indicating whether topology cache is enabled. Default value is "false".
-#gg.client.cacheTop=false
+#ignite.client.cacheTop=false
 
 # Optional. Max time of connection idleness (ms). Default value is "30000".
-#gg.client.idleTimeout=30000
+#ignite.client.idleTimeout=30000
 
 # Optional. Client protocol, one of TCP or HTTP. Default value is "TCP".
-#gg.client.protocol=TCP
+#ignite.client.protocol=TCP
 
 # Optional. TCP_NODELAY communication flag. Default value is "true".
-#gg.client.tcp.noDelay=true
+#ignite.client.tcp.noDelay=true
 
 # Optional. Topology refresh frequency (ms). Default value is "2000".
-#gg.client.topology.refresh=2000
+#ignite.client.topology.refresh=2000
 
 #
 # Data configurations.
 #
 
 # Optional. List of comma-separated names of data configurations. Default value is "".
-#gg.client.data.configurations=cfg1, cfg2
+#ignite.client.data.configurations=cfg1, cfg2
 
 # Optional. Cache name (space) to work with (for configuration 'cfg1').
-#gg.client.data.cfg1.name=
+#ignite.client.data.cfg1.name=
 
 # Optional. Specific load balancer for configuration 'cfg1'. Default value is "".
 # Balancer may be "random", "roundrobin" or full class name for "your.balancer.ImplementationClass"
-#gg.client.data.cfg1.balancer=random
+#ignite.client.data.cfg1.balancer=random
 
 # Optional. Specific affinity for configuration 'cfg1'. Default value is "".
 # Affinity may be "" (no affinity), "partitioned" or full class name for "your.affinity.ImplementationClass"
-#gg.client.data.cfg1.affinity=
+#ignite.client.data.cfg1.affinity=
 
 # Optional. Cache name (space) to work with (for configuration 'cfg2').
-#gg.client.data.cfg2.name=partitioned
+#ignite.client.data.cfg2.name=partitioned
 
 # Optional. Specific load balancer for configuration 'cfg2'. Default value is "".
 # Balancer may be "random", "roundrobin" or full class name for "your.balancer.ImplementationClass"
-#gg.client.data.cfg2.balancer=roundrobin
+#ignite.client.data.cfg2.balancer=roundrobin
 
 # Optional. Specific affinity for configuration 'cfg2'. Default value is "".
 # Affinity may be "" (no affinity), "partitioned" or full class name for "your.affinity.ImplementationClass"
-#gg.client.data.cfg2.affinity=partitioned
+#ignite.client.data.cfg2.affinity=partitioned
 
 #
 # SSL configuration.
 #
 
 # Optional. SSL enabled. Default value is "false".
-#gg.client.ssl.enabled=false
+#ignite.client.ssl.enabled=false
 
 # Optional. SSL protocol. Default value is "TLS".
-#gg.client.ssl.protocol=TLS
+#ignite.client.ssl.protocol=TLS
 
 # Optional. Key manager algorithm. Default value is "SunX509".
-#gg.client.ssl.key.algorithm=SunX509
+#ignite.client.ssl.key.algorithm=SunX509
 
 # Optional. Keystore to be used by client to connect with Ignite topology over SSL. Default value is "".
-#gg.client.ssl.keystore.location=
+#ignite.client.ssl.keystore.location=
 
 # Optional. Default value is "".
-#gg.client.ssl.keystore.password=
+#ignite.client.ssl.keystore.password=
 
 # Optional. Default value is "jks".
-#gg.client.ssl.keystore.type=jks
+#ignite.client.ssl.keystore.type=jks
 
 # Optional. Truststore to be used by client to connect with Ignite topology over SSL. Default value is "".
-#gg.client.ssl.truststore.location=
+#ignite.client.ssl.truststore.location=
 
 # Optional. Default value is "".
-#gg.client.ssl.truststore.password=
+#ignite.client.ssl.truststore.password=
 
 # Optional. Default value is "jks".
-#gg.client.ssl.truststore.type=jks
+#ignite.client.ssl.truststore.type=jks

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
index 6e2a1eb..8e981d1 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
@@ -85,8 +85,8 @@ public class ClientPropertiesConfigurationSelfTest extends GridCommonAbstractTes
         for (Map.Entry<Object, Object> e : props.entrySet())
             props2.put("new." + e.getKey(), e.getValue());
 
-        validateConfig(0, new GridClientConfiguration("new.gg.client", props2));
-        validateConfig(0, new GridClientConfiguration("new.gg.client.", props2));
+        validateConfig(0, new GridClientConfiguration("new.ignite.client", props2));
+        validateConfig(0, new GridClientConfiguration("new.ignite.client.", props2));
 
         // Validate loaded test configuration.
         File tmp = uncommentProperties(GRID_CLIENT_CONFIG);
@@ -100,14 +100,14 @@ public class ClientPropertiesConfigurationSelfTest extends GridCommonAbstractTes
         for (Map.Entry<Object, Object> e : props.entrySet())
             props2.put("new." + e.getKey(), e.getValue());
 
-        validateConfig(2, new GridClientConfiguration("new.gg.client", props2));
-        validateConfig(2, new GridClientConfiguration("new.gg.client.", props2));
+        validateConfig(2, new GridClientConfiguration("new.ignite.client", props2));
+        validateConfig(2, new GridClientConfiguration("new.ignite.client.", props2));
 
         // Validate loaded test configuration with empty key prefixes.
         props2 = new Properties();
 
         for (Map.Entry<Object, Object> e : props.entrySet())
-            props2.put(e.getKey().toString().replace("gg.client.", ""), e.getValue());
+            props2.put(e.getKey().toString().replace("ignite.client.", ""), e.getValue());
 
         validateConfig(2, new GridClientConfiguration("", props2));
         validateConfig(2, new GridClientConfiguration(".", props2));
@@ -156,7 +156,7 @@ public class ClientPropertiesConfigurationSelfTest extends GridCommonAbstractTes
         Collection<String> lines = new ArrayList<>();
 
         while (it.hasNext())
-            lines.add(it.nextLine().replace("#gg.client.", "gg.client."));
+            lines.add(it.nextLine().replace("#ignite.client.", "ignite.client."));
 
         IgniteUtils.closeQuiet(in);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
index a7c2eae..17ec221 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
@@ -90,101 +90,101 @@ import java.util.logging.*;
  *         <th>Optional</th>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.protocol</b></td>
+ *         <td><b>ignite.client.protocol</b></td>
  *         <td>Communication protocol ({@code TCP} or {@code HTTP}).</td>
  *         <td>{@code TCP}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.connectTimeout</b></td>
+ *         <td><b>ignite.client.connectTimeout</b></td>
  *         <td>Socket connection timeout.</td>
  *         <td>{@code 0} (infinite timeout)</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.tcp.noDelay</b></td>
+ *         <td><b>ignite.client.tcp.noDelay</b></td>
  *         <td>Flag indicating whether TCP_NODELAY flag should be enabled for outgoing connections.</td>
  *         <td>{@code true}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.enabled</b></td>
+ *         <td><b>ignite.client.ssl.enabled</b></td>
  *         <td>Flag indicating that {@code SSL} is needed for connection.</td>
  *         <td>{@code false}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.protocol</b></td>
+ *         <td><b>ignite.client.ssl.protocol</b></td>
  *         <td>SSL protocol ({@code SSL} or {@code TLS}).</td>
  *         <td>{@code TLS}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.key.algorithm</b></td>
+ *         <td><b>ignite.client.ssl.key.algorithm</b></td>
  *         <td>Key manager algorithm.</td>
  *         <td>{@code SunX509}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.keystore.location</b></td>
+ *         <td><b>ignite.client.ssl.keystore.location</b></td>
  *         <td>Key store to be used by client to connect with Ignite topology.</td>
  *         <td>&nbsp;</td>
  *         <td>No (if {@code SSL} is enabled)</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.keystore.password</b></td>
+ *         <td><b>ignite.client.ssl.keystore.password</b></td>
  *         <td>Key store password.</td>
  *         <td>&nbsp;</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.keystore.type</b></td>
+ *         <td><b>ignite.client.ssl.keystore.type</b></td>
  *         <td>Key store type.</td>
  *         <td>{@code jks}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.truststore.location</b></td>
+ *         <td><b>ignite.client.ssl.truststore.location</b></td>
  *         <td>Trust store to be used by client to connect with Ignite topology.</td>
  *         <td>&nbsp;</td>
  *         <td>No (if {@code SSL} is enabled)</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.truststore.password</b></td>
+ *         <td><b>ignite.client.ssl.truststore.password</b></td>
  *         <td>Trust store password.</td>
  *         <td>&nbsp;</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.truststore.type</b></td>
+ *         <td><b>ignite.client.ssl.truststore.type</b></td>
  *         <td>Trust store type.</td>
  *         <td>{@code jks}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.credentials</b></td>
+ *         <td><b>ignite.client.credentials</b></td>
  *         <td>Client credentials used in authentication process.</td>
  *         <td>&nbsp;</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.cache.top</b></td>
+ *         <td><b>ignite.client.cache.top</b></td>
  *         <td>
  *             Flag indicating that topology is cached internally. Cache will be refreshed in
- *             the background with interval defined by {@code gg.client.topology.refresh}
+ *             the background with interval defined by {@code ignite.client.topology.refresh}
  *             property (see below).
  *         </td>
  *         <td>{@code false}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.topology.refresh</b></td>
+ *         <td><b>ignite.client.topology.refresh</b></td>
  *         <td>Topology cache refresh frequency (ms).</td>
  *         <td>{@code 2000}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.idleTimeout</b></td>
+ *         <td><b>ignite.client.idleTimeout</b></td>
  *         <td>Maximum amount of time that connection can be idle before it is closed (ms).</td>
  *         <td>{@code 30000}</td>
  *         <td>Yes</td>
@@ -225,7 +225,7 @@ import java.util.logging.*;
 @SuppressWarnings("JavadocReference")
 public class IgniteJdbcDriver implements Driver {
     /** Prefix for property names. */
-    private static final String PROP_PREFIX = "gg.jdbc.";
+    private static final String PROP_PREFIX = "ignite.jdbc.";
 
     /** Hostname property name. */
     public static final String PROP_HOST = PROP_PREFIX + "host";
@@ -236,11 +236,8 @@ public class IgniteJdbcDriver implements Driver {
     /** Cache name property name. */
     public static final String PROP_CACHE = PROP_PREFIX + "cache";
 
-    /** Node ID URL parameter name. */
-    public static final String PARAM_NODE_ID = "nodeId";
-
     /** Node ID property name. */
-    public static final String PROP_NODE_ID = PROP_PREFIX + PARAM_NODE_ID;
+    public static final String PROP_NODE_ID = PROP_PREFIX + "nodeId";
 
     /** URL prefix. */
     public static final String URL_PREFIX = "jdbc:ignite://";
@@ -290,40 +287,40 @@ public class IgniteJdbcDriver implements Driver {
         props[1] = new PropertyInfo("Port number", info.getProperty(PROP_PORT), "");
         props[2] = new PropertyInfo("Cache name", info.getProperty(PROP_CACHE), "");
         props[3] = new PropertyInfo("Node ID", info.getProperty(PROP_NODE_ID, ""));
-        props[4] = new PropertyInfo("gg.client.protocol", info.getProperty("gg.client.protocol", "TCP"),
+        props[4] = new PropertyInfo("ignite.client.protocol", info.getProperty("ignite.client.protocol", "TCP"),
             "Communication protocol (TCP or HTTP).");
-        props[5] = new PropertyInfo("gg.client.connectTimeout", info.getProperty("gg.client.connectTimeout", "0"),
+        props[5] = new PropertyInfo("ignite.client.connectTimeout", info.getProperty("ignite.client.connectTimeout", "0"),
             "Socket connection timeout.");
-        props[6] = new PropertyInfo("gg.client.tcp.noDelay", info.getProperty("gg.client.tcp.noDelay", "true"),
+        props[6] = new PropertyInfo("ignite.client.tcp.noDelay", info.getProperty("ignite.client.tcp.noDelay", "true"),
             "Flag indicating whether TCP_NODELAY flag should be enabled for outgoing connections.");
-        props[7] = new PropertyInfo("gg.client.ssl.enabled", info.getProperty("gg.client.ssl.enabled", "false"),
+        props[7] = new PropertyInfo("ignite.client.ssl.enabled", info.getProperty("ignite.client.ssl.enabled", "false"),
             "Flag indicating that SSL is needed for connection.");
-        props[8] = new PropertyInfo("gg.client.ssl.protocol", info.getProperty("gg.client.ssl.protocol", "TLS"),
+        props[8] = new PropertyInfo("ignite.client.ssl.protocol", info.getProperty("ignite.client.ssl.protocol", "TLS"),
             "SSL protocol.");
-        props[9] = new PropertyInfo("gg.client.ssl.key.algorithm", info.getProperty("gg.client.ssl.key.algorithm",
+        props[9] = new PropertyInfo("ignite.client.ssl.key.algorithm", info.getProperty("ignite.client.ssl.key.algorithm",
             "SunX509"), "Key manager algorithm.");
-        props[10] = new PropertyInfo("gg.client.ssl.keystore.location",
-            info.getProperty("gg.client.ssl.keystore.location", ""),
+        props[10] = new PropertyInfo("ignite.client.ssl.keystore.location",
+            info.getProperty("ignite.client.ssl.keystore.location", ""),
             "Key store to be used by client to connect with Ignite topology.");
-        props[11] = new PropertyInfo("gg.client.ssl.keystore.password",
-            info.getProperty("gg.client.ssl.keystore.password", ""), "Key store password.");
-        props[12] = new PropertyInfo("gg.client.ssl.keystore.type", info.getProperty("gg.client.ssl.keystore.type",
+        props[11] = new PropertyInfo("ignite.client.ssl.keystore.password",
+            info.getProperty("ignite.client.ssl.keystore.password", ""), "Key store password.");
+        props[12] = new PropertyInfo("ignite.client.ssl.keystore.type", info.getProperty("ignite.client.ssl.keystore.type",
             "jks"), "Key store type.");
-        props[13] = new PropertyInfo("gg.client.ssl.truststore.location",
-            info.getProperty("gg.client.ssl.truststore.location", ""),
+        props[13] = new PropertyInfo("ignite.client.ssl.truststore.location",
+            info.getProperty("ignite.client.ssl.truststore.location", ""),
             "Trust store to be used by client to connect with Ignite topology.");
-        props[14] = new PropertyInfo("gg.client.ssl.keystore.password",
-            info.getProperty("gg.client.ssl.truststore.password", ""), "Trust store password.");
-        props[15] = new PropertyInfo("gg.client.ssl.truststore.type", info.getProperty("gg.client.ssl.truststore.type",
+        props[14] = new PropertyInfo("ignite.client.ssl.keystore.password",
+            info.getProperty("ignite.client.ssl.truststore.password", ""), "Trust store password.");
+        props[15] = new PropertyInfo("ignite.client.ssl.truststore.type", info.getProperty("ignite.client.ssl.truststore.type",
             "jks"), "Trust store type.");
-        props[16] = new PropertyInfo("gg.client.credentials", info.getProperty("gg.client.credentials", ""),
+        props[16] = new PropertyInfo("ignite.client.credentials", info.getProperty("ignite.client.credentials", ""),
             "Client credentials used in authentication process.");
-        props[17] = new PropertyInfo("gg.client.cache.top", info.getProperty("gg.client.cache.top", "false"),
+        props[17] = new PropertyInfo("ignite.client.cache.top", info.getProperty("ignite.client.cache.top", "false"),
             "Flag indicating that topology is cached internally. Cache will be refreshed in the background with " +
                 "interval defined by topologyRefreshFrequency property (see below).");
-        props[18] = new PropertyInfo("gg.client.topology.refresh", info.getProperty("gg.client.topology.refresh",
+        props[18] = new PropertyInfo("ignite.client.topology.refresh", info.getProperty("ignite.client.topology.refresh",
             "2000"), "Topology cache refresh frequency (ms).");
-        props[19] = new PropertyInfo("gg.client.idleTimeout", info.getProperty("gg.client.idleTimeout", "30000"),
+        props[19] = new PropertyInfo("ignite.client.idleTimeout", info.getProperty("ignite.client.idleTimeout", "30000"),
             "Maximum amount of time that connection can be idle before it is closed (ms).");
 
         return props;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
index 3d01afa..ee16f94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
@@ -160,7 +160,7 @@ public class GridClientConfiguration {
      * @throws GridClientException If parsing configuration failed.
      */
     public GridClientConfiguration(Properties in) throws GridClientException {
-        this("gg.client", in);
+        this("ignite.client", in);
     }
 
     /**


[30/50] incubator-ignite git commit: deploy to site disabled

Posted by sb...@apache.org.
deploy to site disabled


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

Branch: refs/heads/ignite-37
Commit: ea91d220928a0ebb215500141d298059d5547320
Parents: 7121619
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:10:29 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:10:29 2015 +0300

----------------------------------------------------------------------
 pom.xml | 46 +++++++++++++++++++++++-----------------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ea91d220/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98541ef..181540b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -577,29 +577,29 @@
                         </executions>
                     </plugin>
 
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-site-plugin</artifactId>
-                        <version>3.4</version>
-                        <dependencies>
-                            <dependency>
-                                <groupId>org.apache.maven.wagon</groupId>
-                                <artifactId>wagon-ssh</artifactId>
-                                <version>2.8</version>
-                            </dependency>
-                        </dependencies>
-                        <executions>
-                            <execution>
-                                <goals>
-                                    <goal>deploy</goal>
-                                </goals>
-                                <phase>deploy</phase>
-                                <configuration>
-                                    <inputDirectory>${basedir}/target/site</inputDirectory>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
+                    <!--<plugin>-->
+                        <!--<groupId>org.apache.maven.plugins</groupId>-->
+                        <!--<artifactId>maven-site-plugin</artifactId>-->
+                        <!--<version>3.4</version>-->
+                        <!--<dependencies>-->
+                            <!--<dependency>-->
+                                <!--<groupId>org.apache.maven.wagon</groupId>-->
+                                <!--<artifactId>wagon-ssh</artifactId>-->
+                                <!--<version>2.8</version>-->
+                            <!--</dependency>-->
+                        <!--</dependencies>-->
+                        <!--<executions>-->
+                            <!--<execution>-->
+                                <!--<goals>-->
+                                    <!--<goal>deploy</goal>-->
+                                <!--</goals>-->
+                                <!--<phase>deploy</phase>-->
+                                <!--<configuration>-->
+                                    <!--<inputDirectory>${basedir}/target/site</inputDirectory>-->
+                                <!--</configuration>-->
+                            <!--</execution>-->
+                        <!--</executions>-->
+                    <!--</plugin>-->
                 </plugins>
             </build>
         </profile>


[07/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-37
Commit: 7814b53ca63325d33d2340563f00b0c5a06355c2
Parents: a554779 5fb7948
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 17:47:15 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 17:47:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheInvokeResult.java     | 24 ++++++++++++---
 .../processors/cache/GridCacheAdapter.java      |  2 +-
 .../processors/cache/GridCacheMapEntry.java     |  4 +--
 .../processors/cache/GridCacheReturn.java       |  5 +--
 .../dht/atomic/GridDhtAtomicCache.java          |  3 +-
 .../local/atomic/GridLocalAtomicCache.java      |  6 ++--
 .../top/GridTopologyCommandHandler.java         |  3 +-
 .../DataStreamerMultiThreadedSelfTest.java      | 19 +++++++++---
 .../GridSwapSpaceSpiAbstractSelfTest.java       |  2 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    | 32 ++++++++++----------
 10 files changed, 63 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[08/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: 45c6e09b8d4c79683336687724d441c1cfad659e
Parents: 7814b53
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 18:19:14 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 18:19:14 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  2 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  2 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java | 15 ++++++-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  2 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  1 +
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java | 41 ++++++++++++++++++++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |  3 ++
 8 files changed, 62 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 2a3f08a..5aa0ac8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -27,7 +27,7 @@ import java.io.*;
 import java.util.concurrent.atomic.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTest {
     /** Allows to change behavior of readExternal method. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index 0b9226f..453812b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.configuration.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
index 95d38e1..49f58f9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -18,9 +18,13 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import javax.cache.*;
+import java.io.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
     /** {@inheritDoc} */
@@ -64,6 +68,13 @@ public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2
 
         readCnt.set(1);
 
-        jcache(3).get(new TestKey(String.valueOf(key)));
+        try {
+            jcache(3).get(new TestKey(String.valueOf(key)));
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
index ca48507..a4e2753 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
@@ -27,7 +27,7 @@ import javax.cache.*;
 import java.io.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 5d53129..c4a5fd5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -46,6 +46,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTest(IgniteStartUpTestSuite.suite());
         suite.addTest(IgniteExternalizableSelfTestSuite.suite());
         suite.addTest(IgniteP2PSelfTestSuite.suite());
+        suite.addTest(IgniteCacheP2pUnmarshallingErrorTestSuit.suite());
 
         suite.addTest(new TestSuite(GridSelfTest.class));
         suite.addTest(new TestSuite(GridProjectionSelfTest.class));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java
new file mode 100644
index 0000000..9b40280
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuit.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Checks behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingErrorTestSuit extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("P2p Unmarshalling Test Suite");
+
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingNearErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingRebalanceErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingTxErrorTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index afb96be..c5f9632 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.configuration.*;
 import javax.cache.*;
 
 /**
- * Check behavior on exception while unmarshalling key.
+ * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45c6e09b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index ce05980..69d7548 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -111,6 +111,9 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
 
         suite.addTestSuite(GridCacheQueryMetricsSelfTest.class);
 
+        //Unmarshallig query test.
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
+
         return suite;
     }
 }


[16/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-37
Commit: b45679c197da2a5233f0d12ddd80c60761d9f4ff
Parents: 1f35f3a 77d092c
Author: avinogradov <av...@gridgain.com>
Authored: Wed Apr 29 14:51:02 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Apr 29 14:51:02 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               | 22 +++---
 .../processors/cache/IgniteCacheProxy.java      | 10 +--
 .../distributed/GridCacheLockAbstractTest.java  | 75 ++++++++++++++++++++
 3 files changed, 88 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[10/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-37
Commit: debdc2beb84ca2a525fddad18094afc12a39be26
Parents: cf6e435 bd3a572
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 18:45:21 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 18:45:21 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |  44 ++++
 .../org/apache/ignite/internal/IgnitionEx.java  | 165 +++++++++++++-
 .../util/spring/IgniteSpringHelper.java         |  54 ++++-
 .../util/spring/IgniteSpringHelperImpl.java     | 217 +++++++++++++++----
 .../IgniteStartFromStreamConfigurationTest.java |  50 +++++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 6 files changed, 487 insertions(+), 45 deletions(-)
----------------------------------------------------------------------



[49/50] incubator-ignite git commit: # ignite-sprint-5: fix license headers

Posted by sb...@apache.org.
# ignite-sprint-5: fix license headers


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

Branch: refs/heads/ignite-37
Commit: 6f1d642167ebb4563a9c9c7b3429d84214b433cd
Parents: 5f8f6f4
Author: Artem Shutak <as...@gridgain.com>
Authored: Tue May 5 18:34:08 2015 +0300
Committer: Artem Shutak <as...@gridgain.com>
Committed: Tue May 5 18:34:08 2015 +0300

----------------------------------------------------------------------
 dev-tools/build.gradle                     | 15 +++++++--------
 dev-tools/src/main/groovy/jiraslurp.groovy | 15 +++++++--------
 2 files changed, 14 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f1d6421/dev-tools/build.gradle
----------------------------------------------------------------------
diff --git a/dev-tools/build.gradle b/dev-tools/build.gradle
index 30ae6b7..b760bc1 100644
--- a/dev-tools/build.gradle
+++ b/dev-tools/build.gradle
@@ -1,13 +1,12 @@
 /*
- * 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
+ * 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
+ *      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,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f1d6421/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
index 332686f..0a876fa 100644
--- a/dev-tools/src/main/groovy/jiraslurp.groovy
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -1,13 +1,12 @@
 /*
- * 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
+ * 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
+ *      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,


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

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


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

Branch: refs/heads/ignite-37
Commit: 0885ac0565c798a9f4a9b82897deb002d19ee47e
Parents: ea91d22 49a9923
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:10:53 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:10:53 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java      | 4 ----
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 4 ----
 .../ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java     | 8 ++++----
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java | 8 ++++++++
 4 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[28/50] incubator-ignite git commit: minor

Posted by sb...@apache.org.
minor


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

Branch: refs/heads/ignite-37
Commit: e0648d68781e3906b66f3c876fef59bc4141454e
Parents: f6ac27a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 17:18:44 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 17:18:44 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0648d68/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
index 09cbd97..0c9f2f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
@@ -275,9 +275,12 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * TODO: IGNITE-587.
      * @throws Exception If failed.
      */
     public void testClientReconnect() throws Exception {
+        fail("ignite-587");
+
         clientsPerSrv = 1;
 
         startServerNodes(3);
@@ -318,9 +321,12 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * TODO: IGNITE-587.
      * @throws Exception If failed.
      */
     public void testClientNodeLeaveOneServer() throws Exception {
+        fail("ignite-587");
+
         startServerNodes(1);
         startClientNodes(1);
 
@@ -430,6 +436,8 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDataExchangeFromClient() throws Exception {
+        fail("ignite-587");
+
         testDataExchange("client-0");
     }
 


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

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


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

Branch: refs/heads/ignite-37
Commit: 49a992339a34e28b29dbbd84aa5dede1614eebb5
Parents: 7121619 e0648d6
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 17:22:06 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 17:22:06 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java      | 4 ----
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 4 ----
 .../ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java     | 8 ++++----
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java | 8 ++++++++
 4 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



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

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-37
Commit: e406987615bef30b759fabe466abdfdab6d70e65
Parents: 957c206 139aa27
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:50:33 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:50:33 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 ++++++++++---------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 .../discovery/tcp/TcpClientDiscoverySpi.java    |   4 -
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 -
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   8 +-
 .../tcp/TcpClientDiscoverySelfTest.java         |   8 +
 pom.xml                                         |  46 +-
 8 files changed, 384 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


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


[36/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-37
Commit: 445c0a61544fc3d05cef77b0c0653b93859e7da7
Parents: e406987 44b00ea
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:53:36 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:53:36 2015 +0300

----------------------------------------------------------------------
 pom.xml | 46 +++++++++++++++++++++++-----------------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


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


[43/50] incubator-ignite git commit: IGNITE-620. Add CI tooling code for patch validation

Posted by sb...@apache.org.
IGNITE-620. Add CI tooling code for patch validation


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

Branch: refs/heads/ignite-37
Commit: 114a8528c6c0030434227a26523e3c16588145c1
Parents: 8c2e03e
Author: Konstantin Boudnik <co...@wandisco.com>
Authored: Thu Mar 26 12:46:19 2015 -0700
Committer: Konstantin Boudnik <co...@wandisco.com>
Committed: Fri May 1 16:49:09 2015 -0700

----------------------------------------------------------------------
 dev-tools/.gitignore                       |   2 +
 dev-tools/build.gradle                     |  46 ++++++++
 dev-tools/src/main/groovy/jiraslurp.groovy | 147 ++++++++++++++++++++++++
 3 files changed, 195 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/114a8528/dev-tools/.gitignore
----------------------------------------------------------------------
diff --git a/dev-tools/.gitignore b/dev-tools/.gitignore
new file mode 100644
index 0000000..3036616
--- /dev/null
+++ b/dev-tools/.gitignore
@@ -0,0 +1,2 @@
+validated-jira.txt
+.gradle

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/114a8528/dev-tools/build.gradle
----------------------------------------------------------------------
diff --git a/dev-tools/build.gradle b/dev-tools/build.gradle
new file mode 100644
index 0000000..30ae6b7
--- /dev/null
+++ b/dev-tools/build.gradle
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+apply plugin: 'groovy'
+
+repositories {
+    mavenCentral()
+}
+
+dependencies {
+    compile 'org.codehaus.groovy:groovy-all:2.2.1'
+}
+
+task help {
+  println '''There are two interfaces to work with JIRA attachment validation tool
+  - to do the batch validation of all latest patch attachments
+     gradle slurp
+  - to grab a single JIRA's latest attachment and run test validation on it
+     JIRA_NUM=INGITE-### gradle patchapply'''
+}
+
+task slurp (dependsOn: 'classes', type: JavaExec) {
+  args (project.buildDir, 'slurp')
+  main = 'jiraslurp'
+  classpath = sourceSets.main.runtimeClasspath
+}
+
+task patchapply (dependsOn: 'classes', type: JavaExec) {
+  args ("JIRA_NUM=${System.getenv('JIRA_NUM')}")
+  main = 'jiraslurp'
+  classpath = sourceSets.main.runtimeClasspath
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/114a8528/dev-tools/src/main/groovy/jiraslurp.groovy
----------------------------------------------------------------------
diff --git a/dev-tools/src/main/groovy/jiraslurp.groovy b/dev-tools/src/main/groovy/jiraslurp.groovy
new file mode 100644
index 0000000..332686f
--- /dev/null
+++ b/dev-tools/src/main/groovy/jiraslurp.groovy
@@ -0,0 +1,147 @@
+/*
+ * 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.
+ */
+/**
+ * Parsing a special filter from Apache Ignite JIRA and picking up latest by ID
+ * attachments to process.
+ */
+final GIT_REPO = "https://git1-us-west.apache.org/repos/asf/incubator-ignite.git"
+final ATTACHMENT_URL = "https://issues.apache.org/jira/secure/attachment"
+final validated_filename = "validated-jira.txt"
+final LAST_SUCCESSFUL_ARTIFACT = "guestAuth/repository/download/Ignite_PatchValidation_Main/.lastSuccessful/$validated_filename"
+
+final def JIRA_CMD = System.getProperty('JIRA_COMMAND', 'jira.sh')
+LinkedHashMap<String, String> jirasAttached = [:]
+
+def readHistory = {
+  final int MAX_HISTORY = 5000
+
+  List validated_list = []
+  def validated = new File(validated_filename)
+  if (validated.exists()) {
+    validated_list = validated.text.split('\n')
+    validated.delete()
+  } else {
+    try {
+      validated_list =
+          new URL("http://204.14.53.152/$LAST_SUCCESSFUL_ARTIFACT").text.split('\n')
+    } catch (Exception e) {
+      println e.getMessage()
+    }
+  }
+  // Let's make sure the preserved history isn't too long
+  if (validated_list.size > MAX_HISTORY) {
+    validated_list = validated_list[validated_list.size-MAX_HISTORY..validated_list.size-1]
+  }
+  validated_list
+}
+
+/**
+ * Accepting the <jira> XML element from JIRA stream
+ * @return <code>null</code> or <code>JIRA-###,latest_attach_id</code>
+ */
+def getLatestAttachment = { jira ->
+  def latestAttr = jira.attachments[0].attachment.list().sort {
+    it.@id.toInteger()
+  }.reverse()[0]
+  String row = null
+  if (latestAttr == null) {
+    println "${jira.key} is in invalid state: patch is not available"
+  } else {
+    row = "${jira.key},${latestAttr.@id}"
+  }
+}
+
+def checkForAttachments = {
+  def JIRA_FILTER =
+      "https://issues.apache.org/jira/sr/jira.issueviews:searchrequest-xml/12330308/SearchRequest-12330308.xml?tempMax=100&field=key&field=attachments"
+  def rss = new XmlSlurper().parse(JIRA_FILTER)
+  List list = readHistory{}
+
+  rss.channel.item.each { jira ->
+    String row = getLatestAttachment (jira)
+    if (row != null && !list.contains(row)) {
+      def pair = row.split(',')
+      jirasAttached.put(pair[0] as String, pair[1] as String)
+      list.add(row)
+    }
+  }
+
+  // Write everything back to persist the list
+  def validated = new File(validated_filename)
+  validated << list.join('\n')
+}
+
+def checkprocess = { process ->
+  process.waitFor()
+  if (process.exitValue() != 0) {
+    println "Return code: " + process.exitValue()
+    println "Errout:\n" + process.err.text
+    assert process.exitValue() == 0 || process.exitValue() == 128
+  }
+}
+
+def create_gitbranch = {jira, attachementURL ->
+  println jira
+  GIT_REPO
+  println "$ATTACHMENT_URL/$attachementURL/"
+
+  def patchFile = new File("${jira}.patch")
+  patchFile << new URL("$ATTACHMENT_URL/$attachementURL/").text
+  checkprocess "git clone --depth 1 $GIT_REPO".execute()
+  checkprocess "git checkout -b sprint-2 origin/sprint-2".execute(null, new File('incubator-ignite'))
+  checkprocess "git am ../${patchFile.name}".execute(null, new File('incubator-ignite'))
+  patchFile.delete()
+}
+
+def JIRA_xml = { jiranum ->
+  "https://issues.apache.org/jira/si/jira.issueviews:issue-xml/$jiranum/${jiranum}.xml"
+}
+
+args.each {
+  println it
+  def parameters = it.split('=')
+
+  if (parameters[0] == 'slurp') {
+    checkForAttachments()
+    // For each ticket with new attachment, let's trigger remove build
+    jirasAttached.each { k, v ->
+      //  Trailing slash is important for download; only need to pass JIRA number
+      println "Triggering the build for: $k = $ATTACHMENT_URL/$v/"
+    }
+  } else if (parameters.length == 2 && parameters[0] == 'JIRA_NUM' && parameters[1] ==~ /\w+-\d+/) {
+    // Extract JIRA rss from the and pass the ticket element into attachment extraction
+    def rss = new XmlSlurper().parse(JIRA_xml(parameters[1]))
+    String row = getLatestAttachment(rss.channel.item)
+    if (row != null) {
+      def pair = row.split(',')
+      create_gitbranch(pair[0], pair[1])
+    }
+  }
+}
+
+/* Workflow:
+  1. download an attachment if JIRA num's set; otherwise get all latest attachments not mentioned in the
+     validated-jira.txt file from the last successful build
+  2. trigger test build(s) parametrised by JIRA no.
+  3. test build will download JIRA's latest attachment and apply it to currently checked out repo;
+     - build will fail with comment on JIRA if that can not apply
+     - build will post error/success comment depends on the test results
+*/
+// TODO
+//   - TC's test job needs to send a comment to JIRA
+//       $JIRA_CMD -a addComment -s https://issues.apache.org/jira -u ignite-ci -p ci-of-1gnit3 --issue IGNITE-495 --comment "Trying latest version of the jira-cli"


[20/50] incubator-ignite git commit: review

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


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

Branch: refs/heads/ignite-37
Commit: 35e90b8aa7d90680625530701ba42a15d1a03808
Parents: d59e4c4
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Apr 29 18:17:16 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Apr 29 18:17:16 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheTtlManager.java  | 6 +++++-
 .../processors/cache/IgniteCacheEntryListenerAbstractTest.java | 2 +-
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35e90b8a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index d8af2b9..546cf57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -73,6 +73,8 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
      * @param entry Entry to remove.
      */
     public void removeTrackedEntry(GridCacheMapEntry entry) {
+        assert Thread.holdsLock(entry);
+
         pendingEntries.remove(new EntryWrapper(entry));
     }
 
@@ -93,7 +95,9 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
 
         GridCacheVersion obsoleteVer = null;
 
-        int size = pendingEntries.sizex();
+        // Make sure that worker thread (e.g. sys pool) or user thread
+        // will not be trapped.
+        int size = Math.min(pendingEntries.sizex(), 1024);
 
         while (!sizeLimited || size-- > 0) {
             EntryWrapper e = pendingEntries.pollFirst();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35e90b8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index 544fe6c..bb449e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -712,7 +712,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         expirePlcCache.put(key, 10);
 
-        U.sleep(500);
+        U.sleep(700);
 
         if (!eagerTtl())
             assertNull(primaryCache(key, cache.getName()).get(key)); // Provoke expire event if eager ttl is disabled.


[12/50] incubator-ignite git commit: timeouts for faster failure detection

Posted by sb...@apache.org.
timeouts for faster failure detection


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

Branch: refs/heads/ignite-37
Commit: da85b1136f6d2ef47f2a4f3e91a2aa6ba746b1b4
Parents: 11f4957
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Apr 28 13:57:05 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Apr 28 13:57:05 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da85b113/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
index 6f41cb0..b0d8927 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
@@ -63,8 +63,8 @@ abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements Discov
     /** Default value for thread priority (value is <tt>10</tt>). */
     public static final int DFLT_THREAD_PRI = 10;
 
-    /** Default heartbeat messages issuing frequency (value is <tt>500ms</tt>). */
-    public static final long DFLT_HEARTBEAT_FREQ = 500;
+    /** Default heartbeat messages issuing frequency (value is <tt>300ms</tt>). */
+    public static final long DFLT_HEARTBEAT_FREQ = 300;
 
     /** Default size of topology snapshots history. */
     public static final int DFLT_TOP_HISTORY_SIZE = 1000;


[48/50] incubator-ignite git commit: # Minor tests refactoring.

Posted by sb...@apache.org.
# Minor tests 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/5f8f6f46
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/5f8f6f46
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/5f8f6f46

Branch: refs/heads/ignite-37
Commit: 5f8f6f46dbba461e43dc23a292a2b0ff764f19dd
Parents: 57ab09e
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue May 5 12:10:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue May 5 12:10:21 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheSwapReloadSelfTest.java      | 20 ++++++++++++++------
 .../cache/IgniteCachePeekModesAbstractTest.java | 15 ++++++++++++---
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |  4 ++--
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  | 11 ++++++++++-
 .../cache/GridCacheOffHeapSelfTest.java         | 11 ++++++++++-
 5 files changed, 48 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
index 78f01f0..7d4eefc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -35,7 +34,6 @@ import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.*;
-import javax.cache.configuration.*;
 import java.util.*;
 import java.util.concurrent.*;
 
@@ -51,6 +49,18 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
+    /**
+     * Creates swap space spi.
+     * @return The swap spi.
+     */
+    protected SwapSpaceSpi spi() {
+        FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
+
+        swap.setWriteBufferSize(1);
+
+        return swap;
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -62,11 +72,9 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(disco);
 
-        FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
-
-        swap.setWriteBufferSize(1);
+        SwapSpaceSpi spi = spi();
 
-        cfg.setSwapSpaceSpi(swap);
+        cfg.setSwapSpaceSpi(spi);
 
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
index c2eb56f..7cd8414 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 
 import javax.cache.*;
@@ -54,11 +55,19 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         return cfg;
     }
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected NearCacheConfiguration nearConfiguration() {
         return null;
@@ -231,7 +240,7 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
             for (Integer key : keys)
                 cache0.put(key, val);
 
-            FileSwapSpaceSpi swap = (FileSwapSpaceSpi)ignite(nodeIdx).configuration().getSwapSpaceSpi();
+            SwapSpaceSpi swap = ignite(nodeIdx).configuration().getSwapSpaceSpi();
 
             Set<Integer> swapKeys = new HashSet<>();
 
@@ -614,7 +623,7 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
      * @return Tuple with primary and backup keys.
      */
     private T2<List<Integer>, List<Integer>> swapKeys(int nodeIdx) {
-        FileSwapSpaceSpi swap = (FileSwapSpaceSpi)ignite(nodeIdx).configuration().getSwapSpaceSpi();
+        SwapSpaceSpi swap = ignite(nodeIdx).configuration().getSwapSpaceSpi();
 
         IgniteSpiCloseableIterator<Integer> it = swap.keyIterator(SPACE_NAME, null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
index abd80e1..d21fb13 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
@@ -49,7 +49,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
     private static final long MAX_ENTRIES = 9000000;
 
     /** Test duration. */
-    private static final long DURATION = 10 * 60 * 1000;
+    private static final long DURATION = 2 * 60 * 1000;
 
     /** Swap context. */
     private final SwapContext swapCtx = new SwapContext();
@@ -73,7 +73,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
     /**
      * @return An SPI instance to test.
      */
-    private SwapSpaceSpi spi() {
+    protected SwapSpaceSpi spi() {
         FileSwapSpaceSpi spi = new FileSwapSpaceSpi();
 
 //        spi.setConcurrencyLevel(N_THREADS);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 226f0fc..0c55551 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -30,6 +30,7 @@ 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.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -79,6 +80,14 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
     /** */
     private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -91,7 +100,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5f8f6f46/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
index 460a64c..2d6855e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.marshaller.optimized.*;
 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.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -65,6 +66,14 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
     /** PeerClassLoadingLocalClassPathExclude enable. */
     private boolean excluded;
 
+    /**
+     * Creates a SwapSpaceSpi.
+     * @return the Spi
+     */
+    protected SwapSpaceSpi spi() {
+        return new FileSwapSpaceSpi();
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -77,7 +86,7 @@ public class GridCacheOffHeapSelfTest extends GridCommonAbstractTest {
 
         cfg.setNetworkTimeout(2000);
 
-        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        cfg.setSwapSpaceSpi(spi());
 
         CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
 


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

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


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

Branch: refs/heads/ignite-37
Commit: f6ac27a35fe379b8024c824f00d3ce3f84d631fb
Parents: da85b11 ee5bae9
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 12:51:56 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 12:51:56 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 RELEASE_NOTES.txt                               |  13 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++
 modules/cloud/pom.xml                           | 106 ++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 +++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 ++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |   2 +-
 .../configuration/CacheConfiguration.java       | 259 +++++++--
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      | 346 +++++++++---
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/GridUpdateNotifier.java     |  66 ++-
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../apache/ignite/internal/IgniteKernal.java    |  83 ++-
 .../org/apache/ignite/internal/IgnitionEx.java  |  17 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |  36 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../processors/cache/IgniteCacheProxy.java      |  10 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 +++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 +
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../distributed/near/GridNearLockRequest.java   |  54 +-
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 ++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 +-
 .../cache/transactions/IgniteTxManager.java     | 117 +++-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 ++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../ignite/internal/util/GridJavaProcess.java   |   4 +
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../visor/cache/VisorCacheStartTask.java        | 155 ++++++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |  12 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  19 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  13 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 ++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 ------
 .../distributed/GridCacheLockAbstractTest.java  |  75 +++
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 +++++
 .../DataStreamerMultiThreadedSelfTest.java      |   2 +
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++
 modules/gce/pom.xml                             |  92 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 +++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 +++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 +++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 ++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 +++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 ++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 ++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 +++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 ++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 +++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 ++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 ++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 +++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 ++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 ++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 +++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 +++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 ++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  36 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 439 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 275 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 213 files changed, 9863 insertions(+), 3609 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6ac27a3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6ac27a3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------


[23/50] incubator-ignite git commit: Merge branch 'ignite-728' into ignite-sprint-4

Posted by sb...@apache.org.
Merge branch 'ignite-728' into ignite-sprint-4


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

Branch: refs/heads/ignite-37
Commit: d8ced90e512a71e8131329bdc37a936e88f55c3a
Parents: 2a176e4 65780ec5
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 15:40:30 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Apr 30 15:40:30 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 168 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 3 files changed, 115 insertions(+), 62 deletions(-)
----------------------------------------------------------------------



[37/50] incubator-ignite git commit: Release procedure changed

Posted by sb...@apache.org.
Release procedure changed


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

Branch: refs/heads/ignite-37
Commit: 8c2e03e43133d3a546e254fe827a2c4ffc79966a
Parents: 445c0a6
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:54:50 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:54:50 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt | 32 ++++++++++++--------------------
 1 file changed, 12 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c2e03e4/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 7618e6c..1562dc4 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -40,35 +40,27 @@ Specify gpg profile at settings.xml. It will be used to sign sources and artifac
    </properties>
 </profile>
 
-Configure access to people.apache.org/~<username>. It necessary to upload sources to site.
-See https://blogs.apache.org/infra/entry/committer_shell_access_to_people for details.
-Specify ignite-site server at settings.xml.
-
-<server>
-   <id>ignite-site</id>
-   <username>username</username>
-   <privateKey>/home/username.ppk</privateKey>
-   <passphrase>passphrase</passphrase>
-</server>
-
 Ensure you have RELEASE (not SNAPSHOT) version at Ignite poms.
 Maven release plugin release:prepare goal can be used to make release tag.
 
-Deploy Ignite release to maven repository and site:
+Deploy Ignite release candidate to maven repository and dev-svn:
 
-mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -Dignite.site=scp://people.apache.org:/home/<username>/public_html -B
+   mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -B
 
-You might need to allow connection to people.apache.org. Just do following at console:
+   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern.
+   Sources package name is fixed. Sources package zip will be created automatically when apache-release profile used.
 
-ssh people.apache.org
+   In case you want to release both fabric and hadoop editions you have to build hadoop first, save /target/bin/*.zip,
+   make "mvn clean" and restore them before deploy step.
 
-and type "yes" + enter.
+   Nexus staging (repository.apache.org) should be closed when mvn deploy finished.
+   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create new folder with name equals to released
+   version at svn root. Copy target/site folder content to svn/ignite/<version> folder and commit.
 
-In case you want to release both fabric and hadoop editions you have to build hadoop first, save /target/bin/*.zip, make "mvn clean" and
-restore them before deploy step.
+Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>.
 
-Start vote based on people.apache.org/~<username>/ignite-version.
-Release nexus staging when version accepted.
+Release nexus staging and copy binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>
+to https://dist.apache.org/repos/dist/release/incubator/ignite/<version> when version accepted.
 
 
 JCache TCK compliance


[18/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-728' into ignite-728

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


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

Branch: refs/heads/ignite-37
Commit: 8ae6b0319db2e92d7316533d9946bba51d2174c1
Parents: b58e1ac ef7d011
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Apr 29 18:01:02 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Apr 29 18:01:02 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |  44 ++++
 .../ignite/internal/GridJobContextImpl.java     |   7 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 165 +++++++++++++-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 ++++-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 ++--
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 ++++++++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 ++
 .../distributed/dht/GridDhtTxPrepareFuture.java |  10 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../distributed/near/GridNearLockRequest.java   |  54 ++---
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 ++++-
 .../cache/transactions/IgniteTxManager.java     | 117 ++++++++--
 .../ignite/internal/util/GridJavaProcess.java   |   4 +
 .../util/spring/IgniteSpringHelper.java         |  54 ++++-
 .../ignite/internal/visor/cache/VisorCache.java |  92 ++++----
 .../visor/node/VisorBasicConfiguration.java     |  11 +
 .../visor/node/VisorNodeDataCollectorJob.java   |   2 +-
 .../internal/GridDiscoveryEventSelfTest.java    |  30 ++-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++++++++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 ++-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 ++++++++++++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   5 +-
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../util/spring/IgniteSpringHelperImpl.java     | 217 +++++++++++++++----
 .../IgniteStartFromStreamConfigurationTest.java |  50 +++++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 .../config/VisorConfigurationCommand.scala      |   1 +
 .../scala/org/apache/ignite/visor/visor.scala   |  11 +
 pom.xml                                         |   5 +-
 40 files changed, 1396 insertions(+), 213 deletions(-)
----------------------------------------------------------------------



[04/50] incubator-ignite git commit: ignite-728 Need to reimplement CREATE-TIME-TTL as eviction policy

Posted by sb...@apache.org.
ignite-728 Need to reimplement CREATE-TIME-TTL as eviction policy


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

Branch: refs/heads/ignite-37
Commit: b58e1ac8001ff11e2faff2a55c61d8955ca73d95
Parents: 5fb7948
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 23 21:26:31 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Apr 27 17:02:54 2015 +0300

----------------------------------------------------------------------
 .../eviction/sorted/SortedEvictionPolicy.java   |   2 +-
 .../processors/cache/GridCacheTtlManager.java   | 164 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 3 files changed, 110 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b58e1ac8/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
index 0065244..7965c97 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
@@ -381,7 +381,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         private static final long serialVersionUID = 0L;
 
         /** Size. */
-        private volatile LongAdder8 size = new LongAdder8();
+        private final LongAdder8 size = new LongAdder8();
 
         /**
          * @param comp Comparator.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b58e1ac8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index 5198b53..d8af2b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -26,7 +26,8 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
 import org.apache.ignite.thread.*;
 
-import java.util.*;
+import org.jetbrains.annotations.*;
+import org.jsr166.*;
 
 /**
  * Eagerly removes expired entries from cache when {@link org.apache.ignite.configuration.CacheConfiguration#isEagerTtl()} flag is set.
@@ -34,14 +35,11 @@ import java.util.*;
 @SuppressWarnings("NakedNotify")
 public class GridCacheTtlManager extends GridCacheManagerAdapter {
     /** Entries pending removal. */
-    private final GridConcurrentSkipListSet<EntryWrapper> pendingEntries = new GridConcurrentSkipListSet<>();
+    private final GridConcurrentSkipListSetEx pendingEntries = new GridConcurrentSkipListSetEx();
 
     /** Cleanup worker thread. */
     private CleanupWorker cleanupWorker;
 
-    /** Sync mutex. */
-    private final Object mux = new Object();
-
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
         if (cctx.kernalContext().isDaemon() || !cctx.config().isEagerTtl())
@@ -68,24 +66,13 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
      * @param entry Entry to add.
      */
     public void addTrackedEntry(GridCacheMapEntry entry) {
-        EntryWrapper wrapper = new EntryWrapper(entry);
-
-        pendingEntries.add(wrapper);
-
-        // If entry is on the first position, notify waiting thread.
-        if (wrapper == pendingEntries.firstx()) {
-            synchronized (mux) {
-                mux.notifyAll();
-            }
-        }
+        pendingEntries.add(new EntryWrapper(entry));
     }
 
     /**
      * @param entry Entry to remove.
      */
     public void removeTrackedEntry(GridCacheMapEntry entry) {
-        // Remove must be called while holding lock on entry before updating expire time.
-        // No need to wake up waiting thread in this case.
         pendingEntries.remove(new EntryWrapper(entry));
     }
 
@@ -97,6 +84,45 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * Expires entries by TTL.
+     *
+     * @param sizeLimited Size limited.
+     */
+    public void expire(boolean sizeLimited) {
+        long now = U.currentTimeMillis();
+
+        GridCacheVersion obsoleteVer = null;
+
+        int size = pendingEntries.sizex();
+
+        while (!sizeLimited || size-- > 0) {
+            EntryWrapper e = pendingEntries.pollFirst();
+
+            if (e == null)
+                break;
+
+            if (e.expireTime > now) {
+                pendingEntries.add(e);
+
+                break;
+            }
+
+            if (obsoleteVer == null)
+                obsoleteVer = cctx.versions().next();
+
+            if (log.isDebugEnabled())
+                log.debug("Trying to remove expired entry from cache: " + e);
+
+            if (e.entry.onTtlExpired(obsoleteVer)) {
+                e.entry.context().cache().removeEntry(e.entry);
+
+                if (e.entry.context().cache().configuration().isStatisticsEnabled())
+                    e.entry.context().cache().metrics0().onEvict();
+            }
+        }
+    }
+
+    /**
      * Entry cleanup worker.
      */
     private class CleanupWorker extends GridWorker {
@@ -110,52 +136,18 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             while (!isCancelled()) {
-                long now = U.currentTimeMillis();
-
-                GridCacheVersion obsoleteVer = null;
-
-                for (Iterator<EntryWrapper> it = pendingEntries.iterator(); it.hasNext(); ) {
-                    EntryWrapper wrapper = it.next();
-
-                    if (wrapper.expireTime <= now) {
-                        if (log.isDebugEnabled())
-                            log.debug("Trying to remove expired entry from cache: " + wrapper);
-
-                        if (obsoleteVer == null)
-                            obsoleteVer = cctx.versions().next();
-
-                        if (wrapper.entry.onTtlExpired(obsoleteVer))
-                            wrapper.entry.context().cache().removeEntry(wrapper.entry);
+                expire(false);
 
-                        if (wrapper.entry.context().cache().configuration().isStatisticsEnabled())
-                            wrapper.entry.context().cache().metrics0().onEvict();
+                EntryWrapper first = pendingEntries.firstx();
 
-                        it.remove();
-                    }
-                    else
-                        break;
-                }
+                if (first != null) {
+                    long waitTime = first.expireTime - U.currentTimeMillis();
 
-                synchronized (mux) {
-                    while (true) {
-                        // Access of the first element must be inside of
-                        // synchronization block, so we don't miss out
-                        // on thread notification events sent from
-                        // 'addTrackedEntry(..)' method.
-                        EntryWrapper first = pendingEntries.firstx();
-
-                        if (first != null) {
-                            long waitTime = first.expireTime - U.currentTimeMillis();
-
-                            if (waitTime > 0)
-                                mux.wait(waitTime);
-                            else
-                                break;
-                        }
-                        else
-                            mux.wait(5000);
-                    }
+                    if (waitTime > 0)
+                        U.sleep(waitTime);
                 }
+                else
+                    U.sleep(500);
             }
         }
     }
@@ -214,4 +206,58 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
             return res;
         }
     }
+
+    /**
+     * Provides additional method {@code #sizex()}. NOTE: Only the following methods supports this addition:
+     * <ul>
+     *     <li>{@code #add()}</li>
+     *     <li>{@code #remove()}</li>
+     *     <li>{@code #pollFirst()}</li>
+     * <ul/>
+     */
+    private static class GridConcurrentSkipListSetEx extends GridConcurrentSkipListSet<EntryWrapper> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Size. */
+        private final LongAdder8 size = new LongAdder8();
+
+        /**
+         * @return Size based on performed operations.
+         */
+        public int sizex() {
+            return size.intValue();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean add(EntryWrapper e) {
+            boolean res = super.add(e);
+
+            assert res;
+
+            size.increment();
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean remove(Object o) {
+            boolean res = super.remove(o);
+
+            if (res)
+                size.decrement();
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public EntryWrapper pollFirst() {
+            EntryWrapper e = super.pollFirst();
+
+            if (e != null)
+                size.decrement();
+
+            return e;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b58e1ac8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index e7c7f9d..a0e45e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1157,6 +1157,8 @@ public class GridCacheUtils {
 
         if (ctx.isNear())
             ctx.near().dht().context().evicts().unwind();
+
+        ctx.ttl().expire(true);
     }
 
     /**
@@ -1166,11 +1168,12 @@ public class GridCacheUtils {
         assert ctx != null;
 
         for (GridCacheContext<K, V> cacheCtx : ctx.cacheContexts()) {
-
             cacheCtx.evicts().unwind();
 
             if (cacheCtx.isNear())
                 cacheCtx.near().dht().context().evicts().unwind();
+
+            cacheCtx.ttl().expire(true);
         }
     }
 


[02/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: db7aa241ac522c9b9720d3c9bd4198fb66b94df2
Parents: 9e4bc10
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 24 19:38:04 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 24 19:38:04 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 15 ++++
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 72 ++++++++------------
 2 files changed, 42 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db7aa241/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 1fe1f50..c9af788 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -315,6 +315,21 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
+            case 30: {
+                GridDhtLockRequest req = (GridDhtLockRequest)msg;
+
+                GridDhtLockResponse res = new GridDhtLockResponse(
+                    ctx.cacheId(),
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    0);
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
             case 34:{
                 GridDhtTxPrepareRequest req = (GridDhtTxPrepareRequest)msg;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db7aa241/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
index 1a42407..06176aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.transactions.*;
 
+import javax.cache.*;
 import java.io.*;
 
 /**
@@ -35,8 +36,14 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
         return CacheAtomicityMode.TRANSACTIONAL;
     }
 
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (!gridName.endsWith("0"))
+            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
+
+        return cfg;
     }
 
     /**
@@ -68,28 +75,9 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
-//        catch (IgniteException e) {
-//            assert X.hasCause(e, IOException.class);
-//        }
-
-        assert readCnt.get() == 0; //ensure we have read count as expected.
-    }
-
-    /**
-     * Sends put with pessimistic lock and handles fail.
-     */
-    protected void failPessimicticOnCommit() {
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-
-            jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
-            tx.commit();
-
-            assert false : "p2p marshalling failed, but error response was not sent";
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
         }
-//        catch (IgniteException e) {
-//            assert X.hasCause(e, IOException.class);
-//        }
 
         assert readCnt.get() == 0; //ensure we have read count as expected.
     }
@@ -98,32 +86,26 @@ public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmars
      * Tests that correct response will be sent to client node in case of unmarshalling failed.
      */
     public void testResponseMessageOnUnmarshallingFailed() {
-//        //GridNearTxPrepareRequest unmarshalling failed test
-//        readCnt.set(2);
-//
-//        failOptimistic();
-//
-//        //GridDhtTxPrepareRequest unmarshalling failed test
-//        readCnt.set(3);
-//
-//        failOptimistic();
-
-//        //GridNearLockRequest unmarshalling failed test
-//        readCnt.set(2);
-//
-//        failPessimictic();
-
-        //? unmarshalling failed test
-        readCnt.set(1000);
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+        //GridNearTxPrepareRequest unmarshalling failed test
+        readCnt.set(2);
 
-            TestKey tstKey = new TestKey(String.valueOf(++key));
-            jcache(0).put(tstKey, "");
-            jcache(0).lock(tstKey).lock();
-        }
+        failOptimistic();
+
+        //GridDhtTxPrepareRequest unmarshalling failed test
+        readCnt.set(3);
 
+        failOptimistic();
 
+        //GridNearLockRequest unmarshalling failed test
+        readCnt.set(2);
 
+        failPessimictic();
 
+        //GridDhtLockRequest unmarshalling failed test
+        readCnt.set(3);
+
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
+        }
     }
 }


[40/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the close method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the close method.


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

Branch: refs/heads/ignite-37
Commit: 601cf09930a16b1bf477d8407f5deed10c1f50c3
Parents: f298d72
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:32:56 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:32:56 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/601cf099/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index bf4080a..f9007a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -499,7 +499,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
         CacheEntryProcessor<K, V, T> entryProcessor, Object... args);
 
     /**
-     * Completely deletes the cache with all its data from the system.
+     * Completely deletes the cache with all its data from the system on all cluster nodes.
      */
     @Override void close();
 


[14/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-646


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

Branch: refs/heads/ignite-37
Commit: eb20e01a62dcbbd24b12f818b24cfa1af6261576
Parents: debdc2b acc731c
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 28 20:32:52 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 28 20:32:52 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |   2 +-
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../ignite/internal/GridJobContextImpl.java     |   7 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |   6 +-
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 +++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |  10 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../distributed/near/GridNearLockRequest.java   |  54 +-
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 +-
 .../cache/transactions/IgniteTxManager.java     | 117 +++-
 .../ignite/internal/util/GridJavaProcess.java   |   4 +
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../ignite/internal/visor/cache/VisorCache.java |  92 ++--
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../visor/node/VisorBasicConfiguration.java     |  11 +
 .../visor/node/VisorNodeDataCollectorJob.java   |   2 +-
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |   2 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 .../internal/GridDiscoveryEventSelfTest.java    |  30 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 +++++
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   5 +-
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  34 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 438 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 286 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 131 files changed, 3445 insertions(+), 2923 deletions(-)
----------------------------------------------------------------------


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


[38/50] incubator-ignite git commit: src-release zip fix

Posted by sb...@apache.org.
src-release zip fix


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

Branch: refs/heads/ignite-37
Commit: 09a33fe39b734f43767fd8ed9bbe2f1275f18324
Parents: 44b00ea
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Apr 30 23:32:41 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Apr 30 23:32:41 2015 +0300

----------------------------------------------------------------------
 pom.xml | 39 ++++++++++++++++++++++++++++++++++-----
 1 file changed, 34 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/09a33fe3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98541ef..c859021 100644
--- a/pom.xml
+++ b/pom.xml
@@ -467,6 +467,35 @@
             <build>
                 <plugins>
                     <plugin>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.apache.resources</groupId>
+                                <artifactId>apache-source-release-assembly-descriptor</artifactId>
+                                <version>1.0.4</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>source-release-assembly</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
+                                    <descriptorRefs>
+                                        <descriptorRef>${sourceReleaseAssemblyDescriptor}</descriptorRef>
+                                    </descriptorRefs>
+                                    <tarLongFileMode>gnu</tarLongFileMode>
+                                    <finalName>incubator-ignite-${project.version}-src</finalName>
+                                    <appendAssemblyId>false</appendAssemblyId>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+
+                    <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-antrun-plugin</artifactId>
                         <executions>
@@ -533,7 +562,7 @@
                                 <fileSet>
                                     <directory>${basedir}/target</directory>
                                     <includes>
-                                        <include>${project.artifactId}-${project.version}-source-release.zip</include>
+                                        <include>incubator-ignite-${project.version}-src.zip</include>
                                         <include>bin/*.zip</include>
                                     </includes>
                                 </fileSet>
@@ -561,10 +590,10 @@
                                     <failOnError>false</failOnError>
                                     <target>
                                         <mkdir dir="${basedir}/target/site" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip" failonerror="false" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip.asc" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.asc" failonerror="false" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip.md5" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.md5" failonerror="false" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip.sha1" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.sha1" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip.asc" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.asc" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip.md5" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.md5" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip.sha1" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.sha1" failonerror="false" />
                                         <copy todir="${basedir}/target/site">
                                             <fileset dir="${basedir}/target/bin">
                                                 <include name="**/*" />


[24/50] incubator-ignite git commit: Apache release fix

Posted by sb...@apache.org.
Apache release fix


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

Branch: refs/heads/ignite-37
Commit: 53996a12effd51bfb5049f8dd42f391d0f2d9d19
Parents: 2a176e4
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 15:54:50 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 15:54:50 2015 +0300

----------------------------------------------------------------------
 pom.xml | 51 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/53996a12/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 03600e6..98541ef 100644
--- a/pom.xml
+++ b/pom.xml
@@ -37,6 +37,7 @@
 
     <properties>
         <ignite.site>scp://localhost:/home</ignite.site>
+        <!--fix <attachartifact>...</> at apache-release profile if changed-->
         <ignite.zip.pattern>ignite-${ignite.edition}-${project.version}-incubating</ignite.zip.pattern>
     </properties>
 
@@ -462,6 +463,55 @@
         </profile>
 
         <profile>
+            <id>apache-release</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>attach-artifact</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                                <configuration>
+                                    <failOnError>false</failOnError>
+                                    <target>
+                                        <attachartifact
+                                            file="${basedir}/target/bin/ignite-fabric-${project.version}-incubating.zip"
+                                            classifier="fabric"
+                                            type="zip"/>
+                                        <attachartifact
+                                            file="${basedir}/target/bin/ignite-hadoop-${project.version}-incubating.zip"
+                                            classifier="hadoop"
+                                            type="zip"/>
+                                    </target>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-gpg-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>sign</goal>
+                                </goals>
+                                <configuration>
+                                    <ascDirectory>${basedir}</ascDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+        <profile>
             <id>deploy-ignite-site</id>
             <build>
                 <plugins>
@@ -484,6 +534,7 @@
                                     <directory>${basedir}/target</directory>
                                     <includes>
                                         <include>${project.artifactId}-${project.version}-source-release.zip</include>
+                                        <include>bin/*.zip</include>
                                     </includes>
                                 </fileSet>
                             </fileSets>


[13/50] incubator-ignite git commit: ignite-728 Need to reimplement CREATE-TIME-TTL as eviction policy

Posted by sb...@apache.org.
ignite-728 Need to reimplement CREATE-TIME-TTL as eviction policy


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

Branch: refs/heads/ignite-37
Commit: ef7d0114c4466eefaff1098c41e5bdb6c3766a28
Parents: 2a68725
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 23 21:26:31 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Apr 28 17:15:45 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 164 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 3 files changed, 111 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef7d0114/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index 5198b53..d8af2b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -26,7 +26,8 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
 import org.apache.ignite.thread.*;
 
-import java.util.*;
+import org.jetbrains.annotations.*;
+import org.jsr166.*;
 
 /**
  * Eagerly removes expired entries from cache when {@link org.apache.ignite.configuration.CacheConfiguration#isEagerTtl()} flag is set.
@@ -34,14 +35,11 @@ import java.util.*;
 @SuppressWarnings("NakedNotify")
 public class GridCacheTtlManager extends GridCacheManagerAdapter {
     /** Entries pending removal. */
-    private final GridConcurrentSkipListSet<EntryWrapper> pendingEntries = new GridConcurrentSkipListSet<>();
+    private final GridConcurrentSkipListSetEx pendingEntries = new GridConcurrentSkipListSetEx();
 
     /** Cleanup worker thread. */
     private CleanupWorker cleanupWorker;
 
-    /** Sync mutex. */
-    private final Object mux = new Object();
-
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
         if (cctx.kernalContext().isDaemon() || !cctx.config().isEagerTtl())
@@ -68,24 +66,13 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
      * @param entry Entry to add.
      */
     public void addTrackedEntry(GridCacheMapEntry entry) {
-        EntryWrapper wrapper = new EntryWrapper(entry);
-
-        pendingEntries.add(wrapper);
-
-        // If entry is on the first position, notify waiting thread.
-        if (wrapper == pendingEntries.firstx()) {
-            synchronized (mux) {
-                mux.notifyAll();
-            }
-        }
+        pendingEntries.add(new EntryWrapper(entry));
     }
 
     /**
      * @param entry Entry to remove.
      */
     public void removeTrackedEntry(GridCacheMapEntry entry) {
-        // Remove must be called while holding lock on entry before updating expire time.
-        // No need to wake up waiting thread in this case.
         pendingEntries.remove(new EntryWrapper(entry));
     }
 
@@ -97,6 +84,45 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * Expires entries by TTL.
+     *
+     * @param sizeLimited Size limited.
+     */
+    public void expire(boolean sizeLimited) {
+        long now = U.currentTimeMillis();
+
+        GridCacheVersion obsoleteVer = null;
+
+        int size = pendingEntries.sizex();
+
+        while (!sizeLimited || size-- > 0) {
+            EntryWrapper e = pendingEntries.pollFirst();
+
+            if (e == null)
+                break;
+
+            if (e.expireTime > now) {
+                pendingEntries.add(e);
+
+                break;
+            }
+
+            if (obsoleteVer == null)
+                obsoleteVer = cctx.versions().next();
+
+            if (log.isDebugEnabled())
+                log.debug("Trying to remove expired entry from cache: " + e);
+
+            if (e.entry.onTtlExpired(obsoleteVer)) {
+                e.entry.context().cache().removeEntry(e.entry);
+
+                if (e.entry.context().cache().configuration().isStatisticsEnabled())
+                    e.entry.context().cache().metrics0().onEvict();
+            }
+        }
+    }
+
+    /**
      * Entry cleanup worker.
      */
     private class CleanupWorker extends GridWorker {
@@ -110,52 +136,18 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             while (!isCancelled()) {
-                long now = U.currentTimeMillis();
-
-                GridCacheVersion obsoleteVer = null;
-
-                for (Iterator<EntryWrapper> it = pendingEntries.iterator(); it.hasNext(); ) {
-                    EntryWrapper wrapper = it.next();
-
-                    if (wrapper.expireTime <= now) {
-                        if (log.isDebugEnabled())
-                            log.debug("Trying to remove expired entry from cache: " + wrapper);
-
-                        if (obsoleteVer == null)
-                            obsoleteVer = cctx.versions().next();
-
-                        if (wrapper.entry.onTtlExpired(obsoleteVer))
-                            wrapper.entry.context().cache().removeEntry(wrapper.entry);
+                expire(false);
 
-                        if (wrapper.entry.context().cache().configuration().isStatisticsEnabled())
-                            wrapper.entry.context().cache().metrics0().onEvict();
+                EntryWrapper first = pendingEntries.firstx();
 
-                        it.remove();
-                    }
-                    else
-                        break;
-                }
+                if (first != null) {
+                    long waitTime = first.expireTime - U.currentTimeMillis();
 
-                synchronized (mux) {
-                    while (true) {
-                        // Access of the first element must be inside of
-                        // synchronization block, so we don't miss out
-                        // on thread notification events sent from
-                        // 'addTrackedEntry(..)' method.
-                        EntryWrapper first = pendingEntries.firstx();
-
-                        if (first != null) {
-                            long waitTime = first.expireTime - U.currentTimeMillis();
-
-                            if (waitTime > 0)
-                                mux.wait(waitTime);
-                            else
-                                break;
-                        }
-                        else
-                            mux.wait(5000);
-                    }
+                    if (waitTime > 0)
+                        U.sleep(waitTime);
                 }
+                else
+                    U.sleep(500);
             }
         }
     }
@@ -214,4 +206,58 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
             return res;
         }
     }
+
+    /**
+     * Provides additional method {@code #sizex()}. NOTE: Only the following methods supports this addition:
+     * <ul>
+     *     <li>{@code #add()}</li>
+     *     <li>{@code #remove()}</li>
+     *     <li>{@code #pollFirst()}</li>
+     * <ul/>
+     */
+    private static class GridConcurrentSkipListSetEx extends GridConcurrentSkipListSet<EntryWrapper> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Size. */
+        private final LongAdder8 size = new LongAdder8();
+
+        /**
+         * @return Size based on performed operations.
+         */
+        public int sizex() {
+            return size.intValue();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean add(EntryWrapper e) {
+            boolean res = super.add(e);
+
+            assert res;
+
+            size.increment();
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean remove(Object o) {
+            boolean res = super.remove(o);
+
+            if (res)
+                size.decrement();
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public EntryWrapper pollFirst() {
+            EntryWrapper e = super.pollFirst();
+
+            if (e != null)
+                size.decrement();
+
+            return e;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef7d0114/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index e7c7f9d..a0e45e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1157,6 +1157,8 @@ public class GridCacheUtils {
 
         if (ctx.isNear())
             ctx.near().dht().context().evicts().unwind();
+
+        ctx.ttl().expire(true);
     }
 
     /**
@@ -1166,11 +1168,12 @@ public class GridCacheUtils {
         assert ctx != null;
 
         for (GridCacheContext<K, V> cacheCtx : ctx.cacheContexts()) {
-
             cacheCtx.evicts().unwind();
 
             if (cacheCtx.isNear())
                 cacheCtx.near().dht().context().evicts().unwind();
+
+            cacheCtx.ttl().expire(true);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef7d0114/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index a873bb0..544fe6c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -77,7 +77,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         super.afterTest();
 
         for (int i = 0; i < gridCount(); i++) {
-            GridContinuousProcessor proc = ((IgniteKernal)grid(i)).context().continuous();
+            GridContinuousProcessor proc = grid(i).context().continuous();
 
             ConcurrentMap<?, ?> syncMsgFuts = GridTestUtils.getFieldValue(proc, "syncMsgFuts");
 
@@ -712,7 +712,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         expirePlcCache.put(key, 10);
 
-        U.sleep(200);
+        U.sleep(500);
 
         if (!eagerTtl())
             assertNull(primaryCache(key, cache.getName()).get(key)); // Provoke expire event if eager ttl is disabled.


[27/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-646' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-646' into ignite-sprint-5


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

Branch: refs/heads/ignite-37
Commit: 957c206bdea433d9cef1a54d0bba0887d51123e8
Parents: d079850 b0374bc
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 16:13:24 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 16:13:24 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 314 +++++++++++++++----
 .../processors/cache/GridCacheMessage.java      |   8 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   8 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   2 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |  18 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   6 +
 .../dht/preloader/GridDhtForceKeysResponse.java |  54 +++-
 .../distributed/near/GridNearGetResponse.java   |   8 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   | 189 +++++++++++
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  56 ++++
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  80 +++++
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 109 +++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |  41 +++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  56 ++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +
 16 files changed, 875 insertions(+), 78 deletions(-)
----------------------------------------------------------------------



[21/50] incubator-ignite git commit: Merge branch 'ignite-sprint-4' into ignite-728

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' into ignite-728


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

Branch: refs/heads/ignite-37
Commit: 65780ec532bcae5e45946aef70b210e3ca6e26a8
Parents: 35e90b8 ee5bae9
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 12:23:55 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Apr 30 12:23:55 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 .../configuration/CacheConfiguration.java       | 255 +++++++++++---
 .../configuration/IgniteConfiguration.java      | 344 +++++++++++++++----
 .../ignite/internal/GridUpdateNotifier.java     |  66 +++-
 .../apache/ignite/internal/IgniteKernal.java    |  83 +++--
 .../org/apache/ignite/internal/IgnitionEx.java  |  15 +-
 .../managers/communication/GridIoManager.java   |   8 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  19 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  13 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 +++++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 ---------
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 12 files changed, 762 insertions(+), 353 deletions(-)
----------------------------------------------------------------------



[50/50] incubator-ignite git commit: ignite-37 Improve offheap metrics for cache

Posted by sb...@apache.org.
ignite-37 Improve offheap metrics for cache


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

Branch: refs/heads/ignite-37
Commit: 3faca8244b393532ae64763fc39a6734a73cad2f
Parents: 6f1d642
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 20:53:53 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed May 6 20:06:34 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheMetrics.java   | 124 +++++++--
 .../processors/cache/CacheMetricsImpl.java      | 162 ++++++++++-
 .../cache/CacheMetricsMXBeanImpl.java           |  55 ++++
 .../processors/cache/CacheMetricsSnapshot.java  | 272 ++++++++++++++-----
 .../processors/cache/GridCacheSwapManager.java  |  75 ++---
 .../offheap/GridOffHeapProcessor.java           |  95 ++++---
 .../ignite/mxbean/CacheMetricsMXBean.java       |  44 +++
 7 files changed, 640 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
index 0d87326..e732d29 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -30,21 +30,21 @@ public interface CacheMetrics {
     /**
      * The number of get requests that were satisfied by the cache.
      *
-     * @return the number of hits
+     * @return The number of hits.
      */
     public long getCacheHits();
 
     /**
      * This is a measure of cache efficiency.
      *
-     * @return the percentage of successful hits, as a decimal e.g 75.
+     * @return The percentage of successful hits, as a decimal e.g 75.
      */
     public float getCacheHitPercentage();
 
     /**
      * A miss is a get request that is not satisfied.
      *
-     * @return the number of misses
+     * @return The number of misses.
      */
     public long getCacheMisses();
 
@@ -52,7 +52,7 @@ public interface CacheMetrics {
      * Returns the percentage of cache accesses that did not find a requested entry
      * in the cache.
      *
-     * @return the percentage of accesses that failed to find anything
+     * @return The percentage of accesses that failed to find anything.
      */
     public float getCacheMissPercentage();
 
@@ -60,14 +60,14 @@ public interface CacheMetrics {
      * The total number of requests to the cache. This will be equal to the sum of
      * the hits and misses.
      *
-     * @return the number of gets
+     * @return The number of gets.
      */
     public long getCacheGets();
 
     /**
      * The total number of puts to the cache.
      *
-     * @return the number of puts
+     * @return The number of puts.
      */
     public long getCachePuts();
 
@@ -75,7 +75,7 @@ public interface CacheMetrics {
      * The total number of removals from the cache. This does not include evictions,
      * where the cache itself initiates the removal to make space.
      *
-     * @return the number of removals
+     * @return The number of removals.
      */
     public long getCacheRemovals();
 
@@ -84,28 +84,28 @@ public interface CacheMetrics {
      * initiated by the cache itself to free up space. An eviction is not treated as
      * a removal and does not appear in the removal counts.
      *
-     * @return the number of evictions
+     * @return The number of evictions.
      */
     public long getCacheEvictions();
 
     /**
      * The mean time to execute gets.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageGetTime();
 
     /**
      * The mean time to execute puts.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAveragePutTime();
 
     /**
      * The mean time to execute removes.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageRemoveTime();
 
@@ -113,7 +113,7 @@ public interface CacheMetrics {
     /**
      * The mean time to execute tx commit.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageTxCommitTime();
 
@@ -124,7 +124,6 @@ public interface CacheMetrics {
      */
     public float getAverageTxRollbackTime();
 
-
     /**
      * Gets total number of transaction commits.
      *
@@ -154,6 +153,62 @@ public interface CacheMetrics {
     public long getOverflowSize();
 
     /**
+     * The total number of get requests to the off-heap memory.
+     *
+     * @return The number of gets.
+     */
+    public long getOffHeapGets();
+
+    /**
+     * The total number of put requests to the off-heap memory.
+     *
+     * @return The number of puts.
+     */
+    public long getOffHeapPuts();
+
+    /**
+     * The total number of removals from the off-heap memory. This does not include evictions.
+     *
+     * @return The number of removals.
+     */
+    public long getOffHeapRemovals();
+
+    /**
+     * The total number of evictions from the off-heap memory.
+     *
+     * @return The number of evictions.
+     */
+    public long getOffHeapEvictions();
+
+    /**
+     * The number of get requests that were satisfied by the off-heap memory.
+     *
+     * @return The off-heap hits number.
+     */
+    public long getOffHeapHits();
+
+    /**
+     * Gets the percentage of hits on off-heap memory.
+     *
+     * @return The percentage of hits on off-heap memory.
+     */
+    public float getOffHeapHitPercentage();
+
+    /**
+     * A miss is a get request that is not satisfied by off-heap memory.
+     *
+     * @return The off-heap misses number.
+     */
+    public long getOffHeapMisses();
+
+    /**
+     * Gets the percentage of misses on off-heap memory.
+     *
+     * @return The percentage of misses on off-heap memory.
+     */
+    public float getOffHeapMissPercentage();
+
+    /**
      * Gets number of entries stored in off-heap memory.
      *
      * @return Number of entries stored in off-heap memory.
@@ -161,6 +216,20 @@ public interface CacheMetrics {
     public long getOffHeapEntriesCount();
 
     /**
+     * Gets number of primary entries stored in off-heap memory.
+     *
+     * @return Number of primary entries stored in off-heap memory.
+     */
+    public long getOffHeapPrimaryEntriesCount();
+
+    /**
+     * Gets number of backup entries stored in off-heap memory.
+     *
+     * @return Number of backup entries stored in off-heap memory.
+     */
+    public long getOffHeapBackupEntriesCount();
+
+    /**
      * Gets memory size allocated in off-heap.
      *
      * @return Memory size allocated in off-heap.
@@ -168,6 +237,13 @@ public interface CacheMetrics {
     public long getOffHeapAllocatedSize();
 
     /**
+     * Gets off-heap memory maximum size.
+     *
+     * @return Off-heap memory maximum size.
+     */
+    public long getOffHeapMaxSize();
+
+    /**
      * Gets number of non-{@code null} values in the cache.
      *
      * @return Number of non-{@code null} values in the cache.
@@ -184,7 +260,7 @@ public interface CacheMetrics {
     /**
      * Returns {@code true} if this cache is empty.
      *
-     * @return {@code true} if this cache is empty.
+     * @return {@code True} if this cache is empty.
      */
     public boolean isEmpty();
 
@@ -294,7 +370,7 @@ public interface CacheMetrics {
     public int getTxDhtRolledbackVersionsSize();
 
     /**
-     * Returns {@code True} if write-behind is enabled.
+     * Returns {@code true} if write-behind is enabled.
      *
      * @return {@code True} if write-behind is enabled.
      */
@@ -372,16 +448,16 @@ public interface CacheMetrics {
     /**
      * Determines the required type of keys for this {@link Cache}, if any.
      *
-     * @return the fully qualified class name of the key type,
-     * or "java.lang.Object" if the type is undefined.
+     * @return The fully qualified class name of the key type,
+     * or {@code "java.lang.Object"} if the type is undefined.
      */
     public String getKeyType();
 
     /**
      * Determines the required type of values for this {@link Cache}, if any.
      *
-     * @return the fully qualified class name of the value type,
-     *         or "java.lang.Object" if the type is undefined.
+     * @return The fully qualified class name of the value type,
+     *         or {@code "java.lang.Object"} if the type is undefined.
      */
     public String getValueType();
 
@@ -407,7 +483,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code true}.
      *
-     * @return true if the cache is store by value
+     * @return {@code True} if the cache is store by value.
      */
     public boolean isStoreByValue();
 
@@ -416,7 +492,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}.
      *
-     * @return true if statistics collection is enabled
+     * @return {@code True} if statistics collection is enabled.
      */
     public boolean isStatisticsEnabled();
 
@@ -425,7 +501,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}.
      *
-     * @return true if management is enabled
+     * @return {@code true} if management is enabled.
      */
     public boolean isManagementEnabled();
 
@@ -434,7 +510,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}
      *
-     * @return {@code true} when a {@link Cache} is in
+     * @return {@code True} when a {@link Cache} is in
      *         "read-through" mode.
      * @see CacheLoader
      */
@@ -448,7 +524,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}
      *
-     * @return {@code true} when a {@link Cache} is in "write-through" mode.
+     * @return {@code True} when a {@link Cache} is in "write-through" mode.
      * @see CacheWriter
      */
     public boolean isWriteThrough();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/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 560de97..c0cbf4b 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
@@ -25,6 +25,8 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.*;
+
 /**
  * Adapter for cache metrics.
  */
@@ -63,7 +65,7 @@ public class CacheMetricsImpl implements CacheMetrics {
     private AtomicLong getTimeNanos = new AtomicLong();
 
     /** Remove time taken nanos. */
-    private AtomicLong removeTimeNanos = new AtomicLong();
+    private AtomicLong rmvTimeNanos = new AtomicLong();
 
     /** Commit transaction time taken nanos. */
     private AtomicLong commitTimeNanos = new AtomicLong();
@@ -71,6 +73,24 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** Commit transaction time taken nanos. */
     private AtomicLong rollbackTimeNanos = new AtomicLong();
 
+    /** Number of reads from off-heap memory. */
+    private AtomicLong offHeapGets = new AtomicLong();
+
+    /** Number of writes to off-heap memory. */
+    private AtomicLong offHeapPuts = new AtomicLong();
+
+    /** Number of removed entries from off-heap memory. */
+    private AtomicLong offHeapRemoves = new AtomicLong();
+
+    /** Number of evictions from off-heap memory. */
+    private AtomicLong offHeapEvicts = new AtomicLong();
+
+    /** Number of off-heap hits. */
+    private AtomicLong offHeapHits = new AtomicLong();
+
+    /** Number of off-heap misses. */
+    private AtomicLong offHeapMisses = new AtomicLong();
+
     /** Cache metrics. */
     @GridToStringExclude
     private transient CacheMetricsImpl delegate;
@@ -126,16 +146,93 @@ public class CacheMetricsImpl implements CacheMetrics {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return offHeapGets.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return offHeapPuts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return offHeapRemoves.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return offHeapEvicts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return offHeapHits.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        long hits0 = offHeapHits.get();
+        long gets0 = offHeapGets.get();
+
+        if (hits0 == 0)
+            return 0;
+
+        return (float) hits0 / gets0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return offHeapMisses.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        long misses0 = offHeapMisses.get();
+        long reads0 = offHeapGets.get();
+
+        if (misses0 == 0)
+            return 0;
+
+        return (float) misses0 / reads0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
         return cctx.cache().offHeapEntriesCount();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        try {
+            return cctx.swap().offheapEntriesCount(true, false, NONE);
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        try {
+            return cctx.swap().offheapEntriesCount(false, true, NONE);
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
         return cctx.cache().offHeapAllocatedSize();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return cctx.config().getOffHeapMaxMemory();
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return cctx.cache().size();
     }
@@ -317,11 +414,18 @@ public class CacheMetricsImpl implements CacheMetrics {
         txCommits.set(0);
         txRollbacks.set(0);
         putTimeNanos.set(0);
-        removeTimeNanos.set(0);
+        rmvTimeNanos.set(0);
         getTimeNanos.set(0);
         commitTimeNanos.set(0);
         rollbackTimeNanos.set(0);
 
+        offHeapGets.set(0);
+        offHeapPuts.set(0);
+        offHeapRemoves.set(0);
+        offHeapHits.set(0);
+        offHeapMisses.set(0);
+        offHeapEvicts.set(0);
+
         if (delegate != null)
             delegate.clear();
     }
@@ -402,7 +506,7 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public float getAverageRemoveTime() {
-        long timeNanos = removeTimeNanos.get();
+        long timeNanos = rmvTimeNanos.get();
         long removesCnt = rmCnt.get();
 
         if (timeNanos == 0 || removesCnt == 0)
@@ -483,7 +587,6 @@ public class CacheMetricsImpl implements CacheMetrics {
             delegate.onTxRollback(duration);
     }
 
-
     /**
      * Increments the get time accumulator.
      *
@@ -514,7 +617,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveTimeNanos(long duration) {
-        removeTimeNanos.addAndGet(duration);
+        rmvTimeNanos.addAndGet(duration);
 
         if (delegate != null)
             delegate.addRemoveTimeNanos(duration);
@@ -526,7 +629,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveAndGetTimeNanos(long duration) {
-        removeTimeNanos.addAndGet(duration);
+        rmvTimeNanos.addAndGet(duration);
         getTimeNanos.addAndGet(duration);
 
         if (delegate != null)
@@ -581,6 +684,53 @@ public class CacheMetricsImpl implements CacheMetrics {
         return cctx.config().isManagementEnabled();
     }
 
+    /**
+     * Off-heap read callback.
+     *
+     * @param hit Hit or miss flag.
+     */
+    public void onOffHeapRead(boolean hit) {
+        offHeapGets.incrementAndGet();
+
+        if (hit)
+            offHeapHits.incrementAndGet();
+        else
+            offHeapMisses.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRead(hit);
+    }
+
+    /**
+     * Off-heap write callback.
+     */
+    public void onOffHeapWrite() {
+        offHeapPuts.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onWrite();
+    }
+
+    /**
+     * Off-heap remove callback.
+     */
+    public void onOffHeapRemove() {
+        offHeapRemoves.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRemove();
+    }
+
+    /**
+     * Off-heap evict callback.
+     */
+    public void onOffHeapEvict() {
+        offHeapEvicts.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRemove();
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheMetricsImpl.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
index e9d547c..127177c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
@@ -49,16 +49,71 @@ class CacheMetricsMXBeanImpl implements CacheMetricsMXBean {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return cache.metrics0().getOffHeapGets();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return cache.metrics0().getOffHeapPuts();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return cache.metrics0().getOffHeapRemovals();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return cache.metrics0().getOffHeapEvictions();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return cache.metrics0().getOffHeapHits();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        return cache.metrics0().getOffHeapHitPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return cache.metrics0().getOffHeapMisses();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        return cache.metrics0().getOffHeapMissPercentage();
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
         return cache.metrics0().getOffHeapEntriesCount();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        return cache.metrics0().getOffHeapPrimaryEntriesCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        return cache.metrics0().getOffHeapBackupEntriesCount();
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
         return cache.metrics0().getOffHeapAllocatedSize();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return cache.metrics0().getOffHeapMaxSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return cache.metrics0().getSize();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
index 4fe152a..34f2f84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
@@ -61,7 +61,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private float getAvgTimeNanos = 0;
 
     /** Remove time taken nanos. */
-    private float removeAvgTimeNanos = 0;
+    private float rmvAvgTimeNanos = 0;
 
     /** Commit transaction time taken nanos. */
     private float commitAvgTimeNanos = 0;
@@ -75,12 +75,39 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     /** Number of entries that was swapped to disk. */
     private long overflowSize;
 
+    /** Number of reads from off-heap. */
+    private long offHeapGets;
+
+    /** Number of writes to off-heap. */
+    private long offHeapPuts;
+
+    /** Number of removed entries from off-heap. */
+    private long offHeapRemoves;
+
+    /** Number of evictions from off-heap. */
+    private long offHeapEvicts;
+
+    /** Off-heap hits number. */
+    private long offHeapHits;
+
+    /** Off-heap misses number. */
+    private long offHeapMisses;
+
     /** Number of entries stored in off-heap memory. */
-    private long offHeapEntriesCount;
+    private long offHeapEntriesCnt;
+
+    /** Number of primary entries stored in off-heap memory. */
+    private long offHeapPrimaryEntriesCnt;
+
+    /** Number of backup entries stored in off-heap memory. */
+    private long offHeapBackupEntriesCnt;
 
     /** Memory size allocated in off-heap. */
     private long offHeapAllocatedSize;
 
+    /** Off-heap memory maximum size*/
+    private long offHeapMaxSize;
+
     /** Number of non-{@code null} values in the cache. */
     private int size;
 
@@ -91,7 +118,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private boolean isEmpty;
 
     /** Gets current size of evict queue used to batch up evictions. */
-    private int dhtEvictQueueCurrentSize;
+    private int dhtEvictQueueCurrSize;
 
     /** Transaction per-thread map size. */
     private int txThreadMapSize;
@@ -106,7 +133,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int txPrepareQueueSize;
 
     /** Start version counts map size. */
-    private int txStartVersionCountsSize;
+    private int txStartVerCountsSize;
 
     /** Number of cached committed transaction IDs. */
     private int txCommittedVersionsSize;
@@ -127,7 +154,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int txDhtPrepareQueueSize;
 
     /** DHT start version counts map size. */
-    private int txDhtStartVersionCountsSize;
+    private int txDhtStartVerCountsSize;
 
     /** Number of cached committed DHT transaction IDs. */
     private int txDhtCommittedVersionsSize;
@@ -142,34 +169,34 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int writeBehindFlushSize;
 
     /** Count of worker threads. */
-    private int writeBehindFlushThreadCount;
+    private int writeBehindFlushThreadCnt;
 
     /** Flush frequency in milliseconds. */
-    private long writeBehindFlushFrequency;
+    private long writeBehindFlushFreq;
 
     /** Maximum size of batch. */
     private int writeBehindStoreBatchSize;
 
     /** Count of cache overflow events since start. */
-    private int writeBehindTotalCriticalOverflowCount;
+    private int writeBehindTotalCriticalOverflowCnt;
 
     /** Count of cache overflow events since start. */
-    private int writeBehindCriticalOverflowCount;
+    private int writeBehindCriticalOverflowCnt;
 
     /** Count of entries in store-retry state. */
-    private int writeBehindErrorRetryCount;
+    private int writeBehindErrorRetryCnt;
 
     /** Total count of entries in cache store internal buffer. */
-    private int writeBehindBufferSize;
+    private int writeBehindBufSize;
 
     /** */
     private String keyType;
 
     /** */
-    private String valueType;
+    private String valType;
 
     /** */
-    private boolean isStoreByValue;
+    private boolean isStoreByVal;
 
     /** */
     private boolean isStatisticsEnabled;
@@ -207,45 +234,54 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         putAvgTimeNanos = m.getAveragePutTime();
         getAvgTimeNanos = m.getAverageGetTime();
-        removeAvgTimeNanos = m.getAverageRemoveTime();
+        rmvAvgTimeNanos = m.getAverageRemoveTime();
         commitAvgTimeNanos = m.getAverageTxCommitTime();
         rollbackAvgTimeNanos = m.getAverageTxRollbackTime();
 
         cacheName = m.name();
         overflowSize = m.getOverflowSize();
-        offHeapEntriesCount = m.getOffHeapEntriesCount();
+        offHeapGets = m.getOffHeapGets();
+        offHeapPuts = m.getOffHeapPuts();
+        offHeapRemoves = m.getOffHeapRemovals();
+        offHeapEvicts = m.getOffHeapEvictions();
+        offHeapHits = m.getOffHeapHits();
+        offHeapMisses = m.getOffHeapMisses();
+        offHeapEntriesCnt = m.getOffHeapEntriesCount();
+        offHeapPrimaryEntriesCnt = m.getOffHeapPrimaryEntriesCount();
+        offHeapBackupEntriesCnt = m.getOffHeapBackupEntriesCount();
         offHeapAllocatedSize = m.getOffHeapAllocatedSize();
+        offHeapMaxSize = m.getOffHeapMaxSize();
         size = m.getSize();
         keySize = m.getKeySize();
         isEmpty = m.isEmpty();
-        dhtEvictQueueCurrentSize = m.getDhtEvictQueueCurrentSize();
+        dhtEvictQueueCurrSize = m.getDhtEvictQueueCurrentSize();
         txThreadMapSize = m.getTxThreadMapSize();
         txXidMapSize = m.getTxXidMapSize();
         txCommitQueueSize = m.getTxCommitQueueSize();
         txPrepareQueueSize = m.getTxPrepareQueueSize();
-        txStartVersionCountsSize = m.getTxStartVersionCountsSize();
+        txStartVerCountsSize = m.getTxStartVersionCountsSize();
         txCommittedVersionsSize = m.getTxCommittedVersionsSize();
         txRolledbackVersionsSize = m.getTxRolledbackVersionsSize();
         txDhtThreadMapSize = m.getTxDhtThreadMapSize();
         txDhtXidMapSize = m.getTxDhtXidMapSize();
         txDhtCommitQueueSize = m.getTxDhtCommitQueueSize();
         txDhtPrepareQueueSize = m.getTxDhtPrepareQueueSize();
-        txDhtStartVersionCountsSize = m.getTxDhtStartVersionCountsSize();
+        txDhtStartVerCountsSize = m.getTxDhtStartVersionCountsSize();
         txDhtCommittedVersionsSize = m.getTxDhtCommittedVersionsSize();
         txDhtRolledbackVersionsSize = m.getTxDhtRolledbackVersionsSize();
         isWriteBehindEnabled = m.isWriteBehindEnabled();
         writeBehindFlushSize = m.getWriteBehindFlushSize();
-        writeBehindFlushThreadCount = m.getWriteBehindFlushThreadCount();
-        writeBehindFlushFrequency = m.getWriteBehindFlushFrequency();
+        writeBehindFlushThreadCnt = m.getWriteBehindFlushThreadCount();
+        writeBehindFlushFreq = m.getWriteBehindFlushFrequency();
         writeBehindStoreBatchSize = m.getWriteBehindStoreBatchSize();
-        writeBehindTotalCriticalOverflowCount = m.getWriteBehindTotalCriticalOverflowCount();
-        writeBehindCriticalOverflowCount = m.getWriteBehindCriticalOverflowCount();
-        writeBehindErrorRetryCount = m.getWriteBehindErrorRetryCount();
-        writeBehindBufferSize = m.getWriteBehindBufferSize();
+        writeBehindTotalCriticalOverflowCnt = m.getWriteBehindTotalCriticalOverflowCount();
+        writeBehindCriticalOverflowCnt = m.getWriteBehindCriticalOverflowCount();
+        writeBehindErrorRetryCnt = m.getWriteBehindErrorRetryCount();
+        writeBehindBufSize = m.getWriteBehindBufferSize();
 
         keyType = m.getKeyType();
-        valueType = m.getValueType();
-        isStoreByValue = m.isStoreByValue();
+        valType = m.getValueType();
+        isStoreByVal = m.isStoreByValue();
         isStatisticsEnabled = m.isStatisticsEnabled();
         isManagementEnabled = m.isManagementEnabled();
         isReadThrough = m.isReadThrough();
@@ -263,21 +299,23 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         isEmpty = loc.isEmpty();
         isWriteBehindEnabled = loc.isWriteBehindEnabled();
         writeBehindFlushSize = loc.getWriteBehindFlushSize();
-        writeBehindFlushThreadCount = loc.getWriteBehindFlushThreadCount();
-        writeBehindFlushFrequency = loc.getWriteBehindFlushFrequency();
+        writeBehindFlushThreadCnt = loc.getWriteBehindFlushThreadCount();
+        writeBehindFlushFreq = loc.getWriteBehindFlushFrequency();
         writeBehindStoreBatchSize = loc.getWriteBehindStoreBatchSize();
-        writeBehindBufferSize = loc.getWriteBehindBufferSize();
+        writeBehindBufSize = loc.getWriteBehindBufferSize();
         size = loc.getSize();
         keySize = loc.getKeySize();
 
         keyType = loc.getKeyType();
-        valueType = loc.getValueType();
-        isStoreByValue = loc.isStoreByValue();
+        valType = loc.getValueType();
+        isStoreByVal = loc.isStoreByValue();
         isStatisticsEnabled = loc.isStatisticsEnabled();
         isManagementEnabled = loc.isManagementEnabled();
         isReadThrough = loc.isReadThrough();
         isWriteThrough = loc.isWriteThrough();
 
+        offHeapMaxSize = loc.getOffHeapMaxSize();
+
         for (CacheMetrics e : metrics) {
             reads += e.getCacheGets();
             puts += e.getCachePuts();
@@ -290,7 +328,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
             putAvgTimeNanos += e.getAveragePutTime();
             getAvgTimeNanos += e.getAverageGetTime();
-            removeAvgTimeNanos += e.getAverageRemoveTime();
+            rmvAvgTimeNanos += e.getAverageRemoveTime();
             commitAvgTimeNanos += e.getAverageTxCommitTime();
             rollbackAvgTimeNanos += e.getAverageTxRollbackTime();
 
@@ -299,19 +337,27 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
             else
                 overflowSize = -1;
 
-            offHeapEntriesCount += e.getOffHeapEntriesCount();
+            offHeapGets += e.getOffHeapGets();
+            offHeapPuts += e.getOffHeapPuts();
+            offHeapRemoves += e.getOffHeapRemovals();
+            offHeapEvicts += e.getOffHeapEvictions();
+            offHeapHits += e.getOffHeapHits();
+            offHeapMisses += e.getOffHeapMisses();
+            offHeapEntriesCnt += e.getOffHeapEntriesCount();
+            offHeapPrimaryEntriesCnt += e.getOffHeapPrimaryEntriesCount();
+            offHeapBackupEntriesCnt += e.getOffHeapBackupEntriesCount();
             offHeapAllocatedSize += e.getOffHeapAllocatedSize();
 
             if (e.getDhtEvictQueueCurrentSize() > -1)
-                dhtEvictQueueCurrentSize += e.getDhtEvictQueueCurrentSize();
+                dhtEvictQueueCurrSize += e.getDhtEvictQueueCurrentSize();
             else
-                dhtEvictQueueCurrentSize = -1;
+                dhtEvictQueueCurrSize = -1;
 
             txThreadMapSize += e.getTxThreadMapSize();
             txXidMapSize += e.getTxXidMapSize();
             txCommitQueueSize += e.getTxCommitQueueSize();
             txPrepareQueueSize += e.getTxPrepareQueueSize();
-            txStartVersionCountsSize += e.getTxStartVersionCountsSize();
+            txStartVerCountsSize += e.getTxStartVersionCountsSize();
             txCommittedVersionsSize += e.getTxCommittedVersionsSize();
             txRolledbackVersionsSize += e.getTxRolledbackVersionsSize();
 
@@ -336,9 +382,9 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
                 txDhtPrepareQueueSize = -1;
 
             if (e.getTxDhtStartVersionCountsSize() > -1)
-                txDhtStartVersionCountsSize += e.getTxDhtStartVersionCountsSize();
+                txDhtStartVerCountsSize += e.getTxDhtStartVersionCountsSize();
             else
-                txDhtStartVersionCountsSize = -1;
+                txDhtStartVerCountsSize = -1;
 
             if (e.getTxDhtCommittedVersionsSize() > -1)
                 txDhtCommittedVersionsSize += e.getTxDhtCommittedVersionsSize();
@@ -351,19 +397,19 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
                 txDhtRolledbackVersionsSize = -1;
 
             if (e.getWriteBehindTotalCriticalOverflowCount() > -1)
-                writeBehindTotalCriticalOverflowCount += e.getWriteBehindTotalCriticalOverflowCount();
+                writeBehindTotalCriticalOverflowCnt += e.getWriteBehindTotalCriticalOverflowCount();
             else
-                writeBehindTotalCriticalOverflowCount = -1;
+                writeBehindTotalCriticalOverflowCnt = -1;
 
             if (e.getWriteBehindCriticalOverflowCount() > -1)
-                writeBehindCriticalOverflowCount += e.getWriteBehindCriticalOverflowCount();
+                writeBehindCriticalOverflowCnt += e.getWriteBehindCriticalOverflowCount();
             else
-                writeBehindCriticalOverflowCount = -1;
+                writeBehindCriticalOverflowCnt = -1;
 
             if (e.getWriteBehindErrorRetryCount() > -1)
-                writeBehindErrorRetryCount += e.getWriteBehindErrorRetryCount();
+                writeBehindErrorRetryCnt += e.getWriteBehindErrorRetryCount();
             else
-                writeBehindErrorRetryCount = -1;
+                writeBehindErrorRetryCnt = -1;
         }
 
         int size = metrics.size();
@@ -371,7 +417,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         if (size > 1) {
             putAvgTimeNanos /= size;
             getAvgTimeNanos /= size;
-            removeAvgTimeNanos /= size;
+            rmvAvgTimeNanos /= size;
             commitAvgTimeNanos /= size;
             rollbackAvgTimeNanos /= size;
         }
@@ -435,7 +481,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public float getAverageRemoveTime() {
-        return removeAvgTimeNanos;
+        return rmvAvgTimeNanos;
     }
 
     /** {@inheritDoc} */
@@ -469,8 +515,63 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return offHeapGets;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return offHeapPuts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return offHeapRemoves;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return offHeapEvicts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return offHeapHits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        if (offHeapHits == 0 || offHeapGets == 0)
+            return 0;
+
+        return (float) offHeapHits / offHeapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return offHeapMisses;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        if (offHeapMisses == 0 || offHeapGets == 0)
+            return 0;
+
+        return (float) offHeapMisses / offHeapGets * 100.0f;
+    }
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
-        return offHeapEntriesCount;
+        return offHeapEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        return offHeapPrimaryEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        return offHeapBackupEntriesCnt;
     }
 
     /** {@inheritDoc} */
@@ -479,6 +580,11 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return offHeapMaxSize;
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return size;
     }
@@ -495,7 +601,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getDhtEvictQueueCurrentSize() {
-        return dhtEvictQueueCurrentSize;
+        return dhtEvictQueueCurrSize;
     }
 
     /** {@inheritDoc} */
@@ -520,7 +626,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getTxStartVersionCountsSize() {
-        return txStartVersionCountsSize;
+        return txStartVerCountsSize;
     }
 
     /** {@inheritDoc} */
@@ -555,7 +661,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getTxDhtStartVersionCountsSize() {
-        return txDhtStartVersionCountsSize;
+        return txDhtStartVerCountsSize;
     }
 
     /** {@inheritDoc} */
@@ -580,12 +686,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindFlushThreadCount() {
-        return writeBehindFlushThreadCount;
+        return writeBehindFlushThreadCnt;
     }
 
     /** {@inheritDoc} */
     @Override public long getWriteBehindFlushFrequency() {
-        return writeBehindFlushFrequency;
+        return writeBehindFlushFreq;
     }
 
     /** {@inheritDoc} */
@@ -595,22 +701,22 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindTotalCriticalOverflowCount() {
-        return writeBehindTotalCriticalOverflowCount;
+        return writeBehindTotalCriticalOverflowCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindCriticalOverflowCount() {
-        return writeBehindCriticalOverflowCount;
+        return writeBehindCriticalOverflowCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindErrorRetryCount() {
-        return writeBehindErrorRetryCount;
+        return writeBehindErrorRetryCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindBufferSize() {
-        return writeBehindBufferSize;
+        return writeBehindBufSize;
     }
 
     /** {@inheritDoc} */
@@ -620,12 +726,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public String getValueType() {
-        return valueType;
+        return valType;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStoreByValue() {
-        return isStoreByValue;
+        return isStoreByVal;
     }
 
     /** {@inheritDoc} */
@@ -666,31 +772,40 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         out.writeFloat(putAvgTimeNanos);
         out.writeFloat(getAvgTimeNanos);
-        out.writeFloat(removeAvgTimeNanos);
+        out.writeFloat(rmvAvgTimeNanos);
         out.writeFloat(commitAvgTimeNanos);
         out.writeFloat(rollbackAvgTimeNanos);
 
         out.writeLong(overflowSize);
-        out.writeLong(offHeapEntriesCount);
+        out.writeLong(offHeapGets);
+        out.writeLong(offHeapPuts);
+        out.writeLong(offHeapRemoves);
+        out.writeLong(offHeapEvicts);
+        out.writeLong(offHeapHits);
+        out.writeLong(offHeapMisses);
+        out.writeLong(offHeapEntriesCnt);
+        out.writeLong(offHeapPrimaryEntriesCnt);
+        out.writeLong(offHeapBackupEntriesCnt);
         out.writeLong(offHeapAllocatedSize);
-        out.writeInt(dhtEvictQueueCurrentSize);
+        out.writeLong(offHeapMaxSize);
+        out.writeInt(dhtEvictQueueCurrSize);
         out.writeInt(txThreadMapSize);
         out.writeInt(txXidMapSize);
         out.writeInt(txCommitQueueSize);
         out.writeInt(txPrepareQueueSize);
-        out.writeInt(txStartVersionCountsSize);
+        out.writeInt(txStartVerCountsSize);
         out.writeInt(txCommittedVersionsSize);
         out.writeInt(txRolledbackVersionsSize);
         out.writeInt(txDhtThreadMapSize);
         out.writeInt(txDhtXidMapSize);
         out.writeInt(txDhtCommitQueueSize);
         out.writeInt(txDhtPrepareQueueSize);
-        out.writeInt(txDhtStartVersionCountsSize);
+        out.writeInt(txDhtStartVerCountsSize);
         out.writeInt(txDhtCommittedVersionsSize);
         out.writeInt(txDhtRolledbackVersionsSize);
-        out.writeInt(writeBehindTotalCriticalOverflowCount);
-        out.writeInt(writeBehindCriticalOverflowCount);
-        out.writeInt(writeBehindErrorRetryCount);
+        out.writeInt(writeBehindTotalCriticalOverflowCnt);
+        out.writeInt(writeBehindCriticalOverflowCnt);
+        out.writeInt(writeBehindErrorRetryCnt);
     }
 
     /** {@inheritDoc} */
@@ -706,30 +821,39 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         putAvgTimeNanos = in.readFloat();
         getAvgTimeNanos = in.readFloat();
-        removeAvgTimeNanos = in.readFloat();
+        rmvAvgTimeNanos = in.readFloat();
         commitAvgTimeNanos = in.readFloat();
         rollbackAvgTimeNanos = in.readFloat();
 
         overflowSize = in.readLong();
-        offHeapEntriesCount = in.readLong();
+        offHeapGets = in.readLong();
+        offHeapPuts = in.readLong();
+        offHeapRemoves = in.readLong();
+        offHeapEvicts = in.readLong();
+        offHeapHits = in.readLong();
+        offHeapMisses = in.readLong();
+        offHeapEntriesCnt = in.readLong();
+        offHeapPrimaryEntriesCnt = in.readLong();
+        offHeapBackupEntriesCnt = in.readLong();
         offHeapAllocatedSize = in.readLong();
-        dhtEvictQueueCurrentSize = in.readInt();
+        offHeapMaxSize = in.readLong();
+        dhtEvictQueueCurrSize = in.readInt();
         txThreadMapSize = in.readInt();
         txXidMapSize = in.readInt();
         txCommitQueueSize = in.readInt();
         txPrepareQueueSize = in.readInt();
-        txStartVersionCountsSize = in.readInt();
+        txStartVerCountsSize = in.readInt();
         txCommittedVersionsSize = in.readInt();
         txRolledbackVersionsSize = in.readInt();
         txDhtThreadMapSize = in.readInt();
         txDhtXidMapSize = in.readInt();
         txDhtCommitQueueSize = in.readInt();
         txDhtPrepareQueueSize = in.readInt();
-        txDhtStartVersionCountsSize = in.readInt();
+        txDhtStartVerCountsSize = in.readInt();
         txDhtCommittedVersionsSize = in.readInt();
         txDhtRolledbackVersionsSize = in.readInt();
-        writeBehindTotalCriticalOverflowCount = in.readInt();
-        writeBehindCriticalOverflowCount = in.readInt();
-        writeBehindErrorRetryCount = in.readInt();
+        writeBehindTotalCriticalOverflowCnt = in.readInt();
+        writeBehindCriticalOverflowCnt = in.readInt();
+        writeBehindErrorRetryCnt = in.readInt();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 6444e37..fbab17b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -121,6 +121,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         warnFirstEvict();
 
                     writeToSwap(part, cctx.toCacheKeyObject(kb), vb);
+
+                    if (cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onOffHeapEvict();
                 }
                 catch (IgniteCheckedException e) {
                     log.error("Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
@@ -395,8 +398,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @return Reconstituted swap entry or {@code null} if entry is obsolete.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private <X extends GridCacheSwapEntry> X swapEntry(X e) throws IgniteCheckedException
-    {
+    @Nullable private <X extends GridCacheSwapEntry> X swapEntry(X e) throws IgniteCheckedException {
         assert e != null;
 
         checkIteratorQueue();
@@ -425,16 +427,14 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         int part = cctx.affinity().partition(key);
 
         // First check off-heap store.
-        if (offheapEnabled)
-            if (offheap.contains(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext())))
-                return true;
+        if (offheapEnabled && offheap.contains(spaceName, part, key, cctx))
+            return true;
 
         if (swapEnabled) {
             assert key != null;
 
-            byte[] valBytes = swapMgr.read(spaceName,
-                new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
-                cctx.deploy().globalLoader());
+            byte[] valBytes = swapMgr.read(spaceName, new SwapKey(key.value(cctx.cacheObjectContext(), false), part,
+                key.valueBytes(cctx.cacheObjectContext())), cctx.deploy().globalLoader());
 
             return valBytes != null;
         }
@@ -444,7 +444,6 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
     /**
      * @param key Key to read.
-     * @param keyBytes Key bytes.
      * @param part Key partition.
      * @param entryLocked {@code True} if cache entry is locked.
      * @param readOffheap Read offheap flag.
@@ -454,7 +453,6 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      */
     @SuppressWarnings({"unchecked"})
     @Nullable private GridCacheSwapEntry read(KeyCacheObject key,
-        byte[] keyBytes,
         int part,
         boolean entryLocked,
         boolean readOffheap,
@@ -479,7 +477,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
             // First check off-heap store.
             if (readOffheap && offheapEnabled) {
-                byte[] bytes = offheap.get(spaceName, part, key, keyBytes);
+                byte[] bytes = offheap.get(spaceName, part, key, cctx);
 
                 if (bytes != null)
                     return swapEntry(unmarshalSwapEntry(bytes));
@@ -491,8 +489,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             assert key != null;
 
             byte[] bytes = swapMgr.read(spaceName,
-                new SwapKey(key.value(cctx.cacheObjectContext(), false), part, keyBytes),
-                cctx.deploy().globalLoader());
+                new SwapKey(key.value(cctx.cacheObjectContext(), false), part,
+                key.valueBytes(cctx.cacheObjectContext())), cctx.deploy().globalLoader());
 
             if (bytes == null && lsnr != null)
                 return lsnr.entry;
@@ -522,7 +520,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try removing from offheap.
         if (offheapEnabled) {
-            byte[] entryBytes = offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+            byte[] entryBytes = offheap.remove(spaceName, part, key, cctx);
 
             if (entryBytes != null) {
                 GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(entryBytes));
@@ -567,8 +565,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @return Value from swap or {@code null}.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private GridCacheSwapEntry readAndRemoveSwap(final KeyCacheObject key,
-        final int part)
+    @Nullable private GridCacheSwapEntry readAndRemoveSwap(final KeyCacheObject key, final int part)
         throws IgniteCheckedException {
         if (!swapEnabled)
             return null;
@@ -649,12 +646,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         if (!offheapEnabled && !swapEnabled)
             return null;
 
-        return read(entry.key(),
-            entry.key().valueBytes(cctx.cacheObjectContext()),
-            entry.partition(),
-            locked,
-            readOffheap,
-            readSwap);
+        return read(entry.key(), entry.partition(), locked, readOffheap, readSwap);
     }
 
     /**
@@ -670,8 +662,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        IgniteBiTuple<Long, Integer> ptr =
-            offheap.valuePointer(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        IgniteBiTuple<Long, Integer> ptr = offheap.valuePointer(spaceName, part, key, cctx);
 
         if (ptr != null) {
             assert ptr.get1() != null;
@@ -700,7 +691,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        return read(key, key.valueBytes(cctx.cacheObjectContext()), part, false, readOffheap, readSwap);
+        return read(key, part, false, readOffheap, readSwap);
     }
 
     /**
@@ -738,7 +729,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                 int part = cctx.affinity().partition(key);
 
                 byte[] entryBytes =
-                    offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    offheap.remove(spaceName, part, key, cctx);
 
                 if (entryBytes != null) {
                     GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(entryBytes));
@@ -880,7 +871,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        return offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        return offheap.removex(spaceName, part, key, cctx);
     }
 
     /**
@@ -904,7 +895,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        offheap.enableEviction(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        offheap.enableEviction(spaceName, part, key, cctx);
     }
 
     /**
@@ -942,10 +933,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try offheap.
         if (offheapEnabled) {
-            byte[] val = offheap.remove(spaceName,
-                part,
-                key.value(cctx.cacheObjectContext(), false),
-                key.valueBytes(cctx.cacheObjectContext()));
+            byte[] val = offheap.remove(spaceName, part, key, cctx);
 
             if (val != null) {
                 if (c != null)
@@ -1004,13 +992,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             keyClsLdrId,
             valClsLdrId);
 
-        if (offheapEnabled) {
-            offheap.put(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()), entry.marshal());
-
-            if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
-                cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,
-                    EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
-        }
+        if (offheapEnabled)
+            offheap.put(spaceName, part, key, entry.marshal(), cctx);
         else if (swapEnabled)
             writeToSwap(part, key, entry.marshal());
 
@@ -1035,15 +1018,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         if (offheapEnabled) {
             for (GridCacheBatchSwapEntry swapEntry : swapped) {
-                offheap.put(spaceName,
-                    swapEntry.partition(),
-                    swapEntry.key(),
-                    swapEntry.key().valueBytes(cctx.cacheObjectContext()),
-                    swapEntry.marshal());
-
-                if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
-                    cctx.events().addEvent(swapEntry.partition(), swapEntry.key(), cctx.nodeId(),
-                        (IgniteUuid)null, null, EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
+                offheap.put(spaceName, swapEntry.partition(), swapEntry.key(), swapEntry.marshal(), cctx);
 
                 if (qryMgr != null)
                     qryMgr.onSwap(swapEntry.key());
@@ -1274,7 +1249,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                     int part = cctx.affinity().partition(key);
 
-                    offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    offheap.removex(spaceName, part, key, cctx);
                 }
                 else
                     it.removeX();
@@ -1592,7 +1567,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                 int part = cctx.affinity().partition(key);
 
-                offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                offheap.removex(spaceName, part, key, cctx);
             }
 
             @Override protected void onClose() throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
index ebedadb..e605ee0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.offheap;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.*;
@@ -32,6 +33,8 @@ import org.jsr166.*;
 
 import java.util.*;
 
+import static org.apache.ignite.events.EventType.*;
+
 /**
  * Manages offheap memory caches.
  */
@@ -97,13 +100,15 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * Ensures that we have {@code keyBytes}.
      *
      * @param key Key.
-     * @param keyBytes Optional key bytes.
-     * @return Key bytes
+     * @param cctx Cache context.
+     * @return Key bytes.
      * @throws IgniteCheckedException If failed.
      */
-    private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws IgniteCheckedException {
+    private byte[] keyBytes(KeyCacheObject key, GridCacheContext cctx) throws IgniteCheckedException {
         assert key != null;
 
+        byte[] keyBytes = key.valueBytes(cctx.cacheObjectContext());
+
         return keyBytes != null ? keyBytes : marsh.marshal(key);
     }
 
@@ -126,15 +131,20 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
-     * @return {@code true} If offheap space contains value for the given key.
+     * @param cctx Cache context.
+     * @return {@code True} If offheap space contains value for the given key.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean contains(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    public boolean contains(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
-        return m != null && m.contains(part, U.hash(key), keyBytes(key, keyBytes));
+        boolean hit = m != null && m.contains(part, U.hash(key), keyBytes(key, cctx));
+
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapRead(hit);
+
+        return hit;
     }
 
     /**
@@ -143,15 +153,20 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
+     * @param cctx Cache context.
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] get(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    @Nullable public byte[] get(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
-        return m == null ? null : m.get(part, U.hash(key), keyBytes(key, keyBytes));
+        byte[] bytes = m == null ? null : m.get(part, U.hash(key), keyBytes(key, cctx));
+
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapRead(bytes != null);
+
+        return bytes;
     }
 
     /**
@@ -162,15 +177,15 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
+     * @param cctx Cache context.
      * @return Tuple where first value is pointer and second is value size.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, Object key,
-        byte[] keyBytes) throws IgniteCheckedException {
+    @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, KeyCacheObject key,
+        GridCacheContext cctx) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
-        return m == null ? null : m.valuePointer(part, U.hash(key), keyBytes(key, keyBytes));
+        return m == null ? null : m.valuePointer(part, U.hash(key), keyBytes(key, cctx));
     }
 
     /**
@@ -179,15 +194,15 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
+     * @param cctx Cache context.
      * @throws IgniteCheckedException If failed.
      */
-    public void enableEviction(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    public void enableEviction(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         if (m != null)
-            m.enableEviction(part, U.hash(key), keyBytes(key, keyBytes));
+            m.enableEviction(part, U.hash(key), keyBytes(key, cctx));
     }
 
     /**
@@ -196,14 +211,13 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
-     * @param ldr Class loader.
-     * @return Value bytes.
+     * @param cctx Cache context.
+     * @param ldr Class loader.  @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public <T> T getValue(@Nullable String spaceName, int part, Object key, byte[] keyBytes,
-        @Nullable ClassLoader ldr) throws IgniteCheckedException {
-        byte[] valBytes = get(spaceName, part, key, keyBytes);
+    @Nullable public <T> T getValue(@Nullable String spaceName, int part, KeyCacheObject key,
+        GridCacheContext cctx, @Nullable ClassLoader ldr) throws IgniteCheckedException {
+        byte[] valBytes = get(spaceName, part, key, cctx);
 
         if (valBytes == null)
             return null;
@@ -217,14 +231,18 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
+     * @param cctx Cache context.
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] remove(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException {
+    @Nullable public byte[] remove(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
+        throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
-        return m == null ? null : m.remove(part, U.hash(key), keyBytes(key, keyBytes));
+        if(cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapRemove();
+
+        return m == null ? null : m.remove(part, U.hash(key), keyBytes(key, cctx));
     }
 
     /**
@@ -233,11 +251,11 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
      * @param valBytes Value bytes.
+     * @param cctx Cache context.
      * @throws IgniteCheckedException If failed.
      */
-    public void put(@Nullable String spaceName, int part, Object key, byte[] keyBytes, byte[] valBytes)
+    public void put(@Nullable String spaceName, int part, KeyCacheObject key, byte[] valBytes, GridCacheContext cctx)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -245,7 +263,14 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
             throw new IgniteCheckedException("Failed to write data to off-heap space, no space registered for name: " +
                 spaceName);
 
-        m.put(part, U.hash(key), keyBytes(key, keyBytes), valBytes);
+        m.put(part, U.hash(key), keyBytes(key, cctx), valBytes);
+
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapWrite();
+
+        if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
+            cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,
+                EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
     }
 
     /**
@@ -254,14 +279,18 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param spaceName Space name.
      * @param part Partition.
      * @param key Key.
-     * @param keyBytes Key bytes.
-     * @return {@code true} If succeeded.
+     * @param cctx Cache context.
+     * @return {@code True} If succeeded.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean removex(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException {
+    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
+        throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
-        return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes));
+        if(cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapRemove();
+
+        return m != null && m.removex(part, U.hash(key), keyBytes(key, cctx));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3faca824/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
index 2ad07b5..4c37682 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
@@ -100,14 +100,58 @@ public interface CacheMetricsMXBean extends CacheStatisticsMXBean, CacheMXBean,
     public long getOverflowSize();
 
     /** {@inheritDoc} */
+    @MXBeanDescription("Number of gets from off-heap memory.")
+    public long getOffHeapGets();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of puts to off-heap memory.")
+    public long getOffHeapPuts();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of removed entries from off-heap memory.")
+    public long getOffHeapRemovals();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of evictions from off-heap memory.")
+    public long getOffHeapEvictions();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of hits on off-heap memory.")
+    public long getOffHeapHits();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Percentage of hits on off-heap memory.")
+    public float getOffHeapHitPercentage();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of misses on off-heap memory.")
+    public long getOffHeapMisses();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Percentage of misses on off-heap memory.")
+    public float getOffHeapMissPercentage();
+
+    /** {@inheritDoc} */
     @MXBeanDescription("Number of entries stored in off-heap memory.")
     public long getOffHeapEntriesCount();
 
     /** {@inheritDoc} */
+    @MXBeanDescription("Number of primary entries stored in off-heap memory.")
+    public long getOffHeapPrimaryEntriesCount();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of backup stored in off-heap memory.")
+    public long getOffHeapBackupEntriesCount();
+
+    /** {@inheritDoc} */
     @MXBeanDescription("Memory size allocated in off-heap.")
     public long getOffHeapAllocatedSize();
 
     /** {@inheritDoc} */
+    @MXBeanDescription("Off-heap memory maximum size.")
+    public long getOffHeapMaxSize();
+
+    /** {@inheritDoc} */
     @MXBeanDescription("Number of non-null values in the cache.")
     public int getSize();
 


[15/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: 1f35f3aae55fff2b6ab634f5e12fc0e52f276e58
Parents: eb20e01
Author: avinogradov <av...@gridgain.com>
Authored: Tue Apr 28 21:09:57 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Tue Apr 28 21:09:57 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/preloader/GridDhtForceKeysResponse.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1f35f3aa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 1d49b34..5c6e24f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -43,7 +43,6 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
     private IgniteUuid miniId;
 
     /** Error. */
-    @GridDirectTransient
     private volatile IgniteCheckedException err;
 
     /** Missed (not found) keys. */


[33/50] incubator-ignite git commit: deploy to site enabled

Posted by sb...@apache.org.
deploy to site enabled


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

Branch: refs/heads/ignite-37
Commit: 04369fe6d02c9fb2d5cc24eb5b0f9c334bbf92a1
Parents: 0885ac0
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:47:19 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:47:19 2015 +0300

----------------------------------------------------------------------
 pom.xml | 46 +++++++++++++++++++++++-----------------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/04369fe6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 181540b..98541ef 100644
--- a/pom.xml
+++ b/pom.xml
@@ -577,29 +577,29 @@
                         </executions>
                     </plugin>
 
-                    <!--<plugin>-->
-                        <!--<groupId>org.apache.maven.plugins</groupId>-->
-                        <!--<artifactId>maven-site-plugin</artifactId>-->
-                        <!--<version>3.4</version>-->
-                        <!--<dependencies>-->
-                            <!--<dependency>-->
-                                <!--<groupId>org.apache.maven.wagon</groupId>-->
-                                <!--<artifactId>wagon-ssh</artifactId>-->
-                                <!--<version>2.8</version>-->
-                            <!--</dependency>-->
-                        <!--</dependencies>-->
-                        <!--<executions>-->
-                            <!--<execution>-->
-                                <!--<goals>-->
-                                    <!--<goal>deploy</goal>-->
-                                <!--</goals>-->
-                                <!--<phase>deploy</phase>-->
-                                <!--<configuration>-->
-                                    <!--<inputDirectory>${basedir}/target/site</inputDirectory>-->
-                                <!--</configuration>-->
-                            <!--</execution>-->
-                        <!--</executions>-->
-                    <!--</plugin>-->
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-site-plugin</artifactId>
+                        <version>3.4</version>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.maven.wagon</groupId>
+                                <artifactId>wagon-ssh</artifactId>
+                                <version>2.8</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>deploy</goal>
+                                </goals>
+                                <phase>deploy</phase>
+                                <configuration>
+                                    <inputDirectory>${basedir}/target/site</inputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
                 </plugins>
             </build>
         </profile>


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

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


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

Branch: refs/heads/ignite-37
Commit: 71216191625c6d3f577c2e6e27bcb2c8f81d9a6b
Parents: 53996a1 d8ced90
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 15:55:15 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 15:55:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 168 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 3 files changed, 115 insertions(+), 62 deletions(-)
----------------------------------------------------------------------



[11/50] incubator-ignite git commit: timeouts for faster failure detection

Posted by sb...@apache.org.
timeouts for faster failure detection


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

Branch: refs/heads/ignite-37
Commit: 11f4957f90edde2369afe0db8e221f27b53d11b5
Parents: 51dc6aa
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Apr 28 13:56:06 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Apr 28 13:56:06 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java      | 4 ----
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 4 ----
 .../ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java     | 8 ++++----
 3 files changed, 4 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/11f4957f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
index e3182c4..4196306 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
@@ -226,10 +226,6 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
         if (netTimeout < 3000)
             U.warn(log, "Network timeout is too low (at least 3000 ms recommended): " + netTimeout);
 
-        // Warn on odd heartbeat frequency.
-        if (hbFreq < 2000)
-            U.warn(log, "Heartbeat frequency is too high (at least 2000 ms recommended): " + hbFreq);
-
         registerMBean(gridName, this, TcpClientDiscoverySpiMBean.class);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/11f4957f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 6a5eb25..5bc9184 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -853,10 +853,6 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
         if (netTimeout < 3000)
             U.warn(log, "Network timeout is too low (at least 3000 ms recommended): " + netTimeout);
 
-        // Warn on odd heartbeat frequency.
-        if (hbFreq < 2000)
-            U.warn(log, "Heartbeat frequency is too high (at least 2000 ms recommended): " + hbFreq);
-
         registerMBean(gridName, this, TcpDiscoverySpiMBean.class);
 
         if (ipFinder instanceof TcpDiscoveryMulticastIpFinder) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/11f4957f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
index 98e048d..6f41cb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
@@ -51,8 +51,8 @@ abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements Discov
     /** Default port to listen (value is <tt>47500</tt>). */
     public static final int DFLT_PORT = 47500;
 
-    /** Default socket operations timeout in milliseconds (value is <tt>2,000ms</tt>). */
-    public static final long DFLT_SOCK_TIMEOUT = 2000;
+    /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT = 200;
 
     /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>5,000ms</tt>). */
     public static final long DFLT_ACK_TIMEOUT = 5000;
@@ -63,8 +63,8 @@ abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements Discov
     /** Default value for thread priority (value is <tt>10</tt>). */
     public static final int DFLT_THREAD_PRI = 10;
 
-    /** Default heartbeat messages issuing frequency (value is <tt>2,000ms</tt>). */
-    public static final long DFLT_HEARTBEAT_FREQ = 2000;
+    /** Default heartbeat messages issuing frequency (value is <tt>500ms</tt>). */
+    public static final long DFLT_HEARTBEAT_FREQ = 500;
 
     /** Default size of topology snapshots history. */
     public static final int DFLT_TOP_HISTORY_SIZE = 1000;


[09/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: cf6e4352b3913c67123a8d2329069373d4b05075
Parents: 45c6e09
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 18:34:49 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 18:34:49 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheP2pUnmarshallingNearErrorTest.java          | 2 +-
 .../cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java         | 4 +---
 2 files changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6e4352/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index 453812b..732d12d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -44,7 +44,7 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() throws InterruptedException {
         //GridCacheEvictionRequest unmarshalling failed test
-        readCnt.set(5);
+        readCnt.set(5); //2 for each put
 
         jcache(0).put(new TestKey(String.valueOf(++key)), "");
         jcache(0).put(new TestKey(String.valueOf(++key)), "");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6e4352/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index c5f9632..b2095a8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -39,9 +39,7 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
     @Override public void testResponseMessageOnUnmarshallingFailed() {
         readCnt.set(Integer.MAX_VALUE);
 
-        TestKey val = new TestKey(String.valueOf(++key));
-
-        jcache(0).put(val, "");
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
 
         //GridCacheQueryRequest unmarshalling failed test
         readCnt.set(1);


[05/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: a8a9d1089d3bf3d6a4adb56a71aede309fa42977
Parents: 163be30
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 17:41:10 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 17:41:10 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  72 +++++++++++-
 .../atomic/GridNearAtomicUpdateResponse.java    |   2 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   6 +
 .../dht/preloader/GridDhtForceKeysResponse.java |  19 ++++
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |  41 ++++---
 .../IgniteCacheP2pUnmarshallingErrorTxTest.java | 111 -------------------
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |  56 ++++++++++
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  69 ++++++++++++
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 109 ++++++++++++++++++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |  58 ++++++++++
 10 files changed, 410 insertions(+), 133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index c9af788..14d6f7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -301,7 +302,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
-    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx, GridIoPolicy plc) {
+    private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx,
+        GridIoPolicy plc) {
         try {
             cctx.io().send(nodeId, res, plc);
         }
@@ -311,10 +313,24 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
-    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException{
+    private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 
         switch (msg.directType()) {
+            case 14: {
+                GridCacheEvictionRequest req = (GridCacheEvictionRequest)msg;
+
+                GridCacheEvictionResponse res = new GridCacheEvictionResponse(
+                    ctx.cacheId(),
+                    req.futureId(),
+                    req.classError() != null
+                );
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
             case 30: {
                 GridDhtLockRequest req = (GridDhtLockRequest)msg;
 
@@ -330,7 +346,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
-            case 34:{
+            case 34: {
                 GridDhtTxPrepareRequest req = (GridDhtTxPrepareRequest)msg;
 
                 GridDhtTxPrepareResponse res = new GridDhtTxPrepareResponse(
@@ -367,13 +383,37 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     nodeId,
                     req.futureVersion());
 
-                res.onError(req.classError());
+                res.error(req.classError());
+
+                sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
+            }
+
+            break;
+
+            case 42: {
+                GridDhtForceKeysRequest req = (GridDhtForceKeysRequest)msg;
+
+                GridDhtForceKeysResponse res = new GridDhtForceKeysResponse(
+                    ctx.cacheId(),
+                    req.futureId(),
+                    req.miniId()
+                );
+
+                res.error(req.classError());
 
                 sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
             }
 
             break;
 
+            case 45: {
+                GridDhtPartitionSupplyMessage req = (GridDhtPartitionSupplyMessage)msg;
+
+                U.error(log, "Supply message cannot be unmarshalled.", req.classError());
+            }
+
+            break;
+
             case 49: {
                 GridNearGetRequest req = (GridNearGetRequest)msg;
 
@@ -390,6 +430,26 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
+            case 50: {
+                GridNearGetResponse res = (GridNearGetResponse)msg;
+
+                GridPartitionedGetFuture fut = (GridPartitionedGetFuture)ctx.mvcc().future(
+                    res.version(), res.futureId());
+
+                if (fut == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to find future for get response [sender=" + nodeId + ", res=" + res + ']');
+
+                    return;
+                }
+
+                res.error(res.classError());
+
+                fut.onResult(nodeId, res);
+            }
+
+            break;
+
             case 51: {
                 GridNearLockRequest req = (GridNearLockRequest)msg;
 
@@ -878,7 +938,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         catch (Error e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, NoClassDefFoundError.class,
                 UnsupportedClassVersionError.class))
-                    cacheMsg.onClassError(new IgniteCheckedException("Failed to load class during unmarshalling: " + e, e));
+                cacheMsg.onClassError(new IgniteCheckedException("Failed to load class during unmarshalling: " + e, e));
             else
                 throw e;
         }
@@ -907,7 +967,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
 
         /** {@inheritDoc} */
-        @SuppressWarnings( {"CatchGenericClass", "unchecked"})
+        @SuppressWarnings({"CatchGenericClass", "unchecked"})
         @Override public void onMessage(final UUID nodeId, Object msg) {
             if (log.isDebugEnabled())
                 log.debug("Received cache ordered message [nodeId=" + nodeId + ", msg=" + msg + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 773b847..330e43c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -140,7 +140,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * Sets update error.
      * @param err
      */
-    public void onError(IgniteCheckedException err){
+    public void error(IgniteCheckedException err){
         this.err = err;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 61aaa14..78966d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -451,6 +451,12 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
          * @param res Result callback.
          */
         void onResult(GridDhtForceKeysResponse res) {
+            if (res.error() != null) {
+                onDone(res.error());
+
+                return;
+            }
+
             Collection<KeyCacheObject> missedKeys = res.missedKeys();
 
             boolean remapMissed = false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 8919185..1d49b34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -42,6 +42,10 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
     /** Mini-future ID. */
     private IgniteUuid miniId;
 
+    /** Error. */
+    @GridDirectTransient
+    private volatile IgniteCheckedException err;
+
     /** Missed (not found) keys. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -73,6 +77,21 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
         this.miniId = miniId;
     }
 
+    /**
+     * Sets error.
+     * @param err
+     */
+    public void error(IgniteCheckedException err){
+        this.err = err;
+    }
+
+    /**
+     * @return Error, if any.
+     */
+    public IgniteCheckedException error() {
+        return err;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean allowForStartup() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 277f10d..b358b15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.annotations.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -82,6 +83,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         }
 
         /** Field. */
+        @QuerySqlField(index = true)
         private String field;
 
         /** {@inheritDoc} */
@@ -110,7 +112,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             field = (String)in.readObject();
 
-            if (readCnt.decrementAndGet() <= 0) { //will throw exception on backup node only
+            if (readCnt.decrementAndGet() <= 0) {
                 throw new IOException("Class can not be unmarshalled");
             }
         }
@@ -119,9 +121,9 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /**
      * Sends put atomically and handles fail.
      */
-    protected void failAtomicPut() {
+    protected void failAtomicPut(int k) {
         try {
-            jcache(0).put(new TestKey("1"), "");
+            jcache(0).put(new TestKey(String.valueOf(k)), "");
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
@@ -135,9 +137,9 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /**
      * Sends get atomically and handles fail.
      */
-    protected void failAtomicGet() {
+    protected void failAtomicGet(int k) {
         try {
-            jcache(0).get(new TestKey("1"));
+            jcache(0).get(new TestKey(String.valueOf(k)));
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
@@ -149,30 +151,39 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /**
      * Tests that correct response will be sent to client node in case of unmarshalling failed.
      */
-    public void testResponseMessageOnUnmarshallingFailed() {
+    public void testResponseMessageOnUnmarshallingFailed() throws Exception {
         //GridNearAtomicUpdateRequest unmarshalling failed test
         readCnt.set(1);
 
-        failAtomicPut();
-
-        //GridNearGetRequest unmarshalling failed test
-        readCnt.set(1);
-
-        failAtomicGet();
+        failAtomicPut(++key);
 
         //Check that cache is empty.
         readCnt.set(100);
 
-        assert jcache(0).get(new TestKey("1")) == null;
+        assert jcache(0).get(new TestKey(String.valueOf(key))) == null;
 
         //GridDhtAtomicUpdateRequest unmarshalling failed test
         readCnt.set(2);
 
-        failAtomicPut();
+        failAtomicPut(++key);
 
         //Check that cache is not empty.
         readCnt.set(100);
 
-        assert jcache(0).get(new TestKey("1")) != null;
+        assert jcache(0).get(new TestKey(String.valueOf(key))) != null;
+
+        //GridNearGetRequest unmarshalling failed test
+        readCnt.set(1);
+
+        failAtomicGet(++key);
+
+        //GridNearGetResponse unmarshalling failed test
+        readCnt.set(100);
+
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+        readCnt.set(2);
+
+        failAtomicGet(key);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
deleted file mode 100644
index 06176aa..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTxTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- *  Licensed to the Apache Software Foundation (ASF) under one or more
- *  contributor license agreements.  See the NOTICE file distributed with
- *  this work for additional information regarding copyright ownership.
- *  The ASF licenses this file to You under the Apache License, Version 2.0
- *  (the "License"); you may not use this file except in compliance with
- *  the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.transactions.*;
-
-import javax.cache.*;
-import java.io.*;
-
-/**
- * Check behavior on exception while unmarshalling key.
- */
-public class IgniteCacheP2pUnmarshallingErrorTxTest extends IgniteCacheP2pUnmarshallingErrorTest {
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return CacheAtomicityMode.TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        if (!gridName.endsWith("0"))
-            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
-
-        return cfg;
-    }
-
-    /**
-     * Sends put with optimistic lock and handles fail.
-     */
-    protected void failOptimistic() {
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-
-            jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
-            tx.commit();
-
-            assert false : "p2p marshalling failed, but error response was not sent";
-        }
-        catch (IgniteException e) {
-            assert X.hasCause(e, IOException.class);
-        }
-
-        assert readCnt.get() == 0; //ensure we have read count as expected.
-    }
-
-    /**
-     * Sends put with pessimistic lock and handles fail.
-     */
-    protected void failPessimictic() {
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-
-            jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
-            assert false : "p2p marshalling failed, but error response was not sent";
-        }
-        catch (CacheException e) {
-            assert X.hasCause(e, IOException.class);
-        }
-
-        assert readCnt.get() == 0; //ensure we have read count as expected.
-    }
-
-    /**
-     * Tests that correct response will be sent to client node in case of unmarshalling failed.
-     */
-    public void testResponseMessageOnUnmarshallingFailed() {
-        //GridNearTxPrepareRequest unmarshalling failed test
-        readCnt.set(2);
-
-        failOptimistic();
-
-        //GridDhtTxPrepareRequest unmarshalling failed test
-        readCnt.set(3);
-
-        failOptimistic();
-
-        //GridNearLockRequest unmarshalling failed test
-        readCnt.set(2);
-
-        failPessimictic();
-
-        //GridDhtLockRequest unmarshalling failed test
-        readCnt.set(3);
-
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
-            jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
new file mode 100644
index 0000000..0b9226f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -0,0 +1,56 @@
+/*
+ *  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.eviction.fifo.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.getCacheConfiguration()[0].setEvictMaxOverflowRatio(0);
+        cfg.getCacheConfiguration()[0].setEvictSynchronized(true);
+        cfg.getCacheConfiguration()[0].setEvictSynchronizedKeyBufferSize(1);
+        cfg.getCacheConfiguration()[0].setEvictionPolicy(new FifoEvictionPolicy(1));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() throws InterruptedException {
+        //GridCacheEvictionRequest unmarshalling failed test
+        readCnt.set(5);
+
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+        jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+        //Eviction request unmarshalling failed but ioManager does not hangs up.
+
+        Thread.sleep(1000); //todo: wait for eviction complete
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
new file mode 100644
index 0000000..f247a00
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -0,0 +1,69 @@
+/*
+ *  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.affinity.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() throws Exception {
+        //GridDhtPartitionSupplyMessage unmarshalling failed test
+        readCnt.set(100);
+
+        for (int i = 0; i <= 20; i++) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+        }
+
+        readCnt.set(1);
+
+        startGrid(3);
+
+        //GridDhtPartitionSupplyMessage unmarshalling failed but ioManager does not hangs up.
+
+        Thread.sleep(1000);
+
+        //GridDhtForceKeysRequest unmarshalling failed test
+        stopGrid(3);
+
+        readCnt.set(Integer.MAX_VALUE);
+
+        for (int i = 0; i <= 1000; i++) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+        }
+
+        startGrid(3);
+
+        Affinity<Object> aff = affinity(grid(3).cache(null));
+
+        while (!aff.isPrimary(grid(3).localNode(), new TestKey(String.valueOf(key)))) {
+            --key;
+        }
+
+        readCnt.set(1);
+
+        jcache(3).get(new TestKey(String.valueOf(key)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
new file mode 100644
index 0000000..ca48507
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
@@ -0,0 +1,109 @@
+/*
+ *  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.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import java.io.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (!gridName.endsWith("0"))
+            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
+
+        return cfg;
+    }
+
+    /**
+     * Sends put with optimistic lock and handles fail.
+     */
+    protected void failOptimistic() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+            tx.commit();
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (IgniteException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert readCnt.get() == 0; //ensure we have read count as expected.
+    }
+
+    /**
+     * Sends put with pessimistic lock and handles fail.
+     */
+    protected void failPessimictic() {
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+
+            jcache(0).put(new TestKey(String.valueOf(++key)), "");
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            assert X.hasCause(e, IOException.class);
+        }
+
+        assert readCnt.get() == 0; //ensure we have read count as expected.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() {
+        //GridNearTxPrepareRequest unmarshalling failed test
+        readCnt.set(2);
+
+        failOptimistic();
+
+        //GridDhtTxPrepareRequest unmarshalling failed test
+        readCnt.set(3);
+
+        failOptimistic();
+
+        //GridNearLockRequest unmarshalling failed test
+        readCnt.set(2);
+
+        failPessimictic();
+
+        //GridDhtLockRequest unmarshalling failed test
+        readCnt.set(3);
+
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+            jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a8a9d108/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
new file mode 100644
index 0000000..265490c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -0,0 +1,58 @@
+/*
+ *  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.query.*;
+import org.apache.ignite.configuration.*;
+
+import javax.cache.*;
+
+/**
+ * Check behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.getCacheConfiguration()[0].setIndexedTypes(TestKey.class, String.class);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testResponseMessageOnUnmarshallingFailed() {
+        readCnt.set(100);
+
+        TestKey val = new TestKey(String.valueOf(++key));
+
+        jcache(0).put(val, "");
+
+        //GridCacheQueryRequest unmarshalling failed test
+        readCnt.set(1);
+
+        try {
+            jcache(0).query(new SqlQuery<TestKey, String>(String.class, "field like '" + key + "'")).getAll();
+
+            assert false : "p2p marshalling failed, but error response was not sent";
+        }
+        catch (CacheException e) {
+            // No-op
+        }
+    }
+}


[41/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the withAsync method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the withAsync method.


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

Branch: refs/heads/ignite-37
Commit: 2cb73176d3a671cb2ac784310a4d36b2d03bad29
Parents: 601cf09
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:45:38 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:45:38 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2cb73176/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
index 76ebcee..f5b5153 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
@@ -22,7 +22,7 @@ package org.apache.ignite.lang;
  */
 public interface IgniteAsyncSupport {
     /**
-     * Gets component with asynchronous mode enabled.
+     * Gets instance of this component with asynchronous mode enabled.
      *
      * @return Component with asynchronous mode enabled.
      */


[47/50] incubator-ignite git commit: Merge branch 'ignite-sprint-4' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-37
Commit: 57ab09ee168d70cbd65266944d263681c51a4f3c
Parents: da3d9e7 54f9492
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue May 5 12:09:09 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue May 5 12:09:09 2015 +0300

----------------------------------------------------------------------
 .../streaming/wordcount/CacheConfig.java        |   5 -
 .../config/grid-client-config.properties        |  50 +-
 .../ClientPropertiesConfigurationSelfTest.java  |  12 +-
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../org/apache/ignite/IgniteJdbcDriver.java     |  81 ++-
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../client/GridClientConfiguration.java         |   2 +-
 .../processors/cache/GridCacheAdapter.java      | 503 +++++++++----------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   4 +-
 pom.xml                                         |  39 +-
 11 files changed, 389 insertions(+), 435 deletions(-)
----------------------------------------------------------------------


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


[44/50] incubator-ignite git commit: Revert "#ignite-732: IgniteCache.size() should not fail in case of topology changes."

Posted by sb...@apache.org.
Revert "#ignite-732: IgniteCache.size() should not fail in case of topology changes."

This reverts commit 139aa270ae61494c0757867f2dc531ec7251b1da.


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

Branch: refs/heads/ignite-37
Commit: c4bc92974bace5e4cdb3ac9dc80790193e46d203
Parents: 281f4ef
Author: ivasilinets <iv...@gridgain.com>
Authored: Sat May 2 10:05:35 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Sat May 2 10:05:35 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 +++++++++----------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 3 files changed, 277 insertions(+), 349 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4bc9297/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
index 87081fc..c2ad198 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
@@ -24,16 +24,15 @@ import java.util.*;
 
 /**
  * Convenience adapter for {@link ComputeTask} interface. Here is an example of
- * how {@code ComputeTaskAdapter} can be used:
+ * how {@code GridComputeTaskAdapter} can be used:
  * <pre name="code" class="java">
- * public class MyFooBarTask extends ComputeTaskAdapter&lt;String, String&gt; {
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
  *     // Inject load balancer.
  *     &#64;LoadBalancerResource
  *     ComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg)
- *         throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteCheckedException {
  *         Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -77,8 +76,8 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * <p>
      * If remote job resulted in exception ({@link ComputeJobResult#getException()} is not {@code null}),
      * then {@link ComputeJobResultPolicy#FAILOVER} policy will be returned if the exception is instance
-     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException},
-     * which means that remote node either failed or job execution was rejected before it got a chance to start. In all
+     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException}, which means that
+     * remote node either failed or job execution was rejected before it got a chance to start. In all
      * other cases the exception will be rethrown which will ultimately cause task to fail.
      *
      * @param res Received remote grid executable result.
@@ -88,8 +87,7 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * @throws IgniteException If handling a job result caused an error effectively rejecting
      *      a failover. This exception will be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd)
-        throws IgniteException {
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
         IgniteException e = res.getException();
 
         // Try to failover if result is failed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4bc9297/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 39f19b1..3f4e97b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -75,9 +75,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Failed result. */
-    private static final Object FAIL = new Integer(-1);
-
     /** clearLocally() split threshold. */
     public static final int CLEAR_ALL_SPLIT_THRESHOLD = 10000;
 
@@ -885,7 +882,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<Cache.Entry<K, V>> entrySet() {
-        return entrySet((CacheEntryPredicate[])null);
+        return entrySet((CacheEntryPredicate[]) null);
     }
 
     /** {@inheritDoc} */
@@ -900,17 +897,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> keySet() {
-        return keySet((CacheEntryPredicate[])null);
+        return keySet((CacheEntryPredicate[]) null);
     }
 
     /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
-        return primaryKeySet((CacheEntryPredicate[])null);
+        return primaryKeySet((CacheEntryPredicate[]) null);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<V> values() {
-        return values((CacheEntryPredicate[])null);
+        return values((CacheEntryPredicate[]) null);
     }
 
     /**
@@ -1083,31 +1080,36 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void clear() throws IgniteCheckedException {
-        clearAll(0, new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
+        // Clear local cache synchronously.
+        clearLocally();
+
+        clearRemotes(0, new GlobalClearAllCallable(name()));
     }
 
     /** {@inheritDoc} */
     @Override public void clear(K key) throws IgniteCheckedException {
-        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            Collections.singleton(key)));
+        // Clear local cache synchronously.
+        clearLocally(key);
+
+        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public void clearAll(Set<? extends K> keys) throws IgniteCheckedException {
-        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            keys));
+        // Clear local cache synchronously.
+        clearLocallyAll(keys);
+
+        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), keys));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(K key) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            Collections.singleton(key)));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(Set<? extends K> keys) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            keys));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), keys));
     }
 
     /**
@@ -1116,13 +1118,19 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param clearCall Global clear callable object.
      * @throws IgniteCheckedException In case of cache could not be cleared on any of the nodes.
      */
-    private void clearAll(long timeout, TopologyVersionAwareCallable clearCall) throws IgniteCheckedException {
+    private void clearRemotes(long timeout, GlobalClearCallable clearCall) throws IgniteCheckedException {
         try {
+            // Send job to remote nodes only.
+            Collection<ClusterNode> nodes =
+                ctx.grid().cluster().forCacheNodes(name(), true, true, false).forRemotes().nodes();
+
             IgniteInternalFuture<Object> fut = null;
 
-            ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
+            if (!nodes.isEmpty()) {
+                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
 
-            fut = new ClearFuture(ctx, clearCall);
+                fut = ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
+            }
 
             if (fut != null)
                 fut.get();
@@ -1141,18 +1149,19 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync() {
-        return clearAsync(new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
+        return clearAsync(new GlobalClearAllCallable(name()));
     }
 
     /**
      * @param clearCall Global clear callable object.
      * @return Future.
      */
-    private IgniteInternalFuture<?> clearAsync(TopologyVersionAwareCallable clearCall) {
+    private IgniteInternalFuture<?> clearAsync(GlobalClearCallable clearCall) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name(), true, true, false).nodes();
 
         if (!nodes.isEmpty()) {
-            IgniteInternalFuture<Object> fut = new ClearFuture(ctx, clearCall);
+            IgniteInternalFuture<Object> fut =
+                ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
 
             return fut.chain(new CX1<IgniteInternalFuture<Object>, Object>() {
                 @Override public Object applyx(IgniteInternalFuture<Object> fut) throws IgniteCheckedException {
@@ -2108,7 +2117,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 throws IgniteCheckedException {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = F.viewAsMap(keys,
                     new C1<K, EntryProcessor<K, V, Object>>() {
-                        @Override public EntryProcessor apply(K k) {
+                            @Override public EntryProcessor apply(K k) {
                             return entryProcessor;
                         }
                     });
@@ -2136,7 +2145,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp() {
             @Override public IgniteInternalFuture<GridCacheReturn> inOp(IgniteTxLocalAdapter tx) {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap =
-                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor);
+                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>) entryProcessor);
 
                 return tx.invokeAsync(ctx, invokeMap, args);
             }
@@ -2362,7 +2371,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.noValArray())
-                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>) RET2VAL);
             }
 
             @Override public String toString() {
@@ -2517,7 +2526,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasValArray()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>) RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2906,7 +2915,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (ctx.deploymentEnabled())
                     ctx.deploy().registerClass(oldVal);
 
-                return tx.putAllAsync(ctx,
+                return (GridCacheReturn) tx.putAllAsync(ctx,
                         F.t(key, newVal),
                         true,
                         null,
@@ -3008,7 +3017,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     ctx.deploy().registerClass(val);
 
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), null, false,
-                    ctx.equalsValArray(val)).get().success();
+                        ctx.equalsValArray(val)).get().success();
             }
 
             @Override public String toString() {
@@ -3221,10 +3230,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         TransactionConfiguration cfg = ctx.gridConfig().getTransactionConfiguration();
 
         return txStart(
-            concurrency,
-            isolation,
-            cfg.getDefaultTxTimeout(),
-            0
+                concurrency,
+                isolation,
+                cfg.getDefaultTxTimeout(),
+                0
         );
     }
 
@@ -3567,7 +3576,22 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0);
 
-        return new SizeFuture(peekModes, ctx, modes.near);
+        IgniteInternalFuture<Collection<Integer>> fut =
+            ctx.closures().broadcastNoFailover(new SizeCallable(ctx.name(), peekModes), null, nodes);
+
+        return fut.chain(new CX1<IgniteInternalFuture<Collection<Integer>>, Integer>() {
+            @Override public Integer applyx(IgniteInternalFuture<Collection<Integer>> fut)
+            throws IgniteCheckedException {
+                Collection<Integer> res = fut.get();
+
+                int totalSize = 0;
+
+                for (Integer size : res)
+                    totalSize += size;
+
+                return totalSize;
+            }
+        });
     }
 
     /** {@inheritDoc} */
@@ -3651,7 +3675,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return F.iterator(iterator(),
             new IgniteClosure<Cache.Entry<K, V>, Cache.Entry<K, V>>() {
                 private IgniteCacheExpiryPolicy expiryPlc =
-                    ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
+                        ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
 
                 @Override public Cache.Entry<K, V> apply(Cache.Entry<K, V> lazyEntry) {
                     CacheOperationContext prev = ctx.gate().enter(opCtx);
@@ -3885,6 +3909,50 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Gets cache global size (with or without backups).
+     *
+     * @param primaryOnly {@code True} if only primary sizes should be included.
+     * @return Global size.
+     * @throws IgniteCheckedException If internal task execution failed.
+     */
+    private int globalSize(boolean primaryOnly) throws IgniteCheckedException {
+        try {
+            // Send job to remote nodes only.
+            Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name()).forRemotes().nodes();
+
+            IgniteInternalFuture<Collection<Integer>> fut = null;
+
+            if (!nodes.isEmpty()) {
+                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, gridCfg.getNetworkTimeout());
+
+                fut = ctx.closures().broadcastNoFailover(new GlobalSizeCallable(name(), primaryOnly), null, nodes);
+            }
+
+            // Get local value.
+            int globalSize = primaryOnly ? primarySize() : size();
+
+            if (fut != null) {
+                for (Integer i : fut.get())
+                    globalSize += i;
+            }
+
+            return globalSize;
+        }
+        catch (ClusterGroupEmptyCheckedException ignore) {
+            if (log.isDebugEnabled())
+                log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
+
+            return primaryOnly ? primarySize() : size();
+        }
+        catch (ComputeTaskTimeoutCheckedException e) {
+            U.warn(log, "Timed out waiting for remote nodes to finish cache clear (consider increasing " +
+                "'networkTimeout' configuration property) [cacheName=" + name() + "]");
+
+            throw e;
+        }
+    }
+
+    /**
      * @param op Cache operation.
      * @param <T> Return type.
      * @return Operation result.
@@ -4825,10 +4893,47 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Internal callable which performs clear operation on a cache with the given name.
+     */
+    @GridInternal
+    private static abstract class GlobalClearCallable implements Callable<Object>, Externalizable {
+        /** Cache name. */
+        protected String cacheName;
+
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        protected Ignite ignite;
+
+        /**
+         * Empty constructor for serialization.
+         */
+        public GlobalClearCallable() {
+            // No-op.
+        }
+
+        /**
+         * @param cacheName Cache name.
+         */
+        protected GlobalClearCallable(String cacheName) {
+            this.cacheName = cacheName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, cacheName);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            cacheName = U.readString(in);
+        }
+    }
+
+    /**
      * Global clear all.
      */
     @GridInternal
-    private static class GlobalClearAllCallable extends TopologyVersionAwareCallable {
+    private static class GlobalClearAllCallable extends GlobalClearCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4841,30 +4946,24 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
          */
-        private GlobalClearAllCallable(String cacheName, AffinityTopologyVersion topVer) {
-            super(cacheName, topVer);
+        private GlobalClearAllCallable(String cacheName) {
+            super(cacheName);
         }
 
         /** {@inheritDoc} */
-        @Override protected Object callLocal() {
+        @Override public Object call() throws Exception {
             ((IgniteEx)ignite).cachex(cacheName).clearLocally();
 
             return null;
         }
-
-        /** {@inheritDoc} */
-        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
-            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
-        }
     }
 
     /**
      * Global clear keys.
      */
     @GridInternal
-    private static class GlobalClearKeySetCallable<K, V> extends TopologyVersionAwareCallable {
+    private static class GlobalClearKeySetCallable<K, V> extends GlobalClearCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4880,25 +4979,33 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
          * @param keys Keys to clear.
          */
-        protected GlobalClearKeySetCallable(String cacheName, AffinityTopologyVersion topVer, Set<? extends K> keys) {
-            super(cacheName, topVer);
+        private GlobalClearKeySetCallable(String cacheName, Set<? extends K> keys) {
+            super(cacheName);
 
             this.keys = keys;
         }
 
         /** {@inheritDoc} */
-        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
-            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
+        @Override public Object call() throws Exception {
+            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
+
+            return null;
         }
 
         /** {@inheritDoc} */
-        @Override protected Object callLocal() {
-            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            super.writeExternal(out);
 
-            return null;
+            out.writeObject(keys);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            super.readExternal(in);
+
+            keys = (Set<K>) in.readObject();
         }
     }
 
@@ -4906,202 +5013,127 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Internal callable for global size calculation.
      */
     @GridInternal
-    private static class GlobalSizeCallable extends TopologyVersionAwareCallable {
+    private static class SizeCallable extends IgniteClosureX<Object, Integer> implements Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
+        /** Cache name. */
+        private String cacheName;
+
         /** Peek modes. */
         private CachePeekMode[] peekModes;
 
-        /** Near enable. */
-        private boolean nearEnable;
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
 
         /**
          * Required by {@link Externalizable}.
          */
-        public GlobalSizeCallable() {
+        public SizeCallable() {
             // No-op.
         }
 
         /**
          * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
          * @param peekModes Cache peek modes.
          */
-        private GlobalSizeCallable(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes, boolean nearEnable) {
-            super(cacheName, topVer);
-
+        private SizeCallable(String cacheName, CachePeekMode[] peekModes) {
+            this.cacheName = cacheName;
             this.peekModes = peekModes;
-            this.nearEnable = nearEnable;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected Object callLocal() {
-            try {
-                IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
-
-                return cache == null ? 0 : cache.localSize(peekModes);
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
         }
 
         /** {@inheritDoc} */
-        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
-            IgniteClusterEx cluster = ctx.grid().cluster();
+        @Override public Integer applyx(Object o) throws IgniteCheckedException {
+            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
-            ClusterGroup grp = nearEnable ? cluster.forCacheNodes(ctx.name(), true, true, false) : cluster.forDataNodes(ctx.name());
+            assert cache != null : cacheName;
 
-            return grp.nodes();
+            return cache.localSize(peekModes);
         }
 
         /** {@inheritDoc} */
-        public String toString() {
-            return S.toString(GlobalSizeCallable.class, this);
-        }
-    }
-
-    /**
-     * Cache size future.
-     */
-    private static class SizeFuture extends RetryFuture {
-        /** Size. */
-        private int size = 0;
-
-        /**
-         * @param peekModes Peek modes.
-         */
-        public SizeFuture(CachePeekMode[] peekModes, GridCacheContext ctx, boolean near) {
-            super(ctx, new GlobalSizeCallable(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes, near));
-        }
+        @SuppressWarnings("ForLoopReplaceableByForEach")
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, cacheName);
 
-        /** {@inheritDoc} */
-        @Override protected void onInit() {
-            size = 0;
-        }
+            out.writeInt(peekModes.length);
 
-        /** {@inheritDoc} */
-        @Override protected void onLocal(Object localRes) {
-            size += (Integer)localRes;
+            for (int i = 0; i < peekModes.length; i++)
+                U.writeEnum(out, peekModes[i]);
         }
 
         /** {@inheritDoc} */
-        @Override protected void allDone() {
-            onDone(size);
-        }
-    }
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            cacheName = U.readString(in);
 
-    /**
-     * Cache clear future.
-     */
-    private static class ClearFuture extends RetryFuture {
-        /**
-         */
-        public ClearFuture(GridCacheContext ctx, TopologyVersionAwareCallable clearCall) {
-            super(ctx, clearCall);
-        }
+            int len = in.readInt();
 
-        /** {@inheritDoc} */
-        @Override protected void onInit() {
-           // No-op.
-        }
+            peekModes = new CachePeekMode[len];
 
-        /** {@inheritDoc} */
-        @Override protected void onLocal(Object localRes) {
-            // No-op.
+            for (int i = 0; i < len; i++)
+                peekModes[i] = CachePeekMode.fromOrdinal(in.readByte());
         }
 
         /** {@inheritDoc} */
-        @Override protected void allDone() {
-            onDone();
+        public String toString() {
+            return S.toString(SizeCallable.class, this);
         }
     }
 
     /**
-     * Retry future.
+     * Internal callable which performs {@link IgniteInternalCache#size()} or {@link IgniteInternalCache#primarySize()}
+     * operation on a cache with the given name.
      */
-    protected static abstract class RetryFuture<T> extends GridFutureAdapter<T> {
-        /** Context. */
-        private final GridCacheContext ctx;
+    @GridInternal
+    private static class GlobalSizeCallable implements IgniteClosure<Object, Integer>, Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
 
-        /** Callable. */
-        private final TopologyVersionAwareCallable call;
+        /** Cache name. */
+        private String cacheName;
 
-        /** Max retries count before issuing an error. */
-        private volatile int retries = 32;
+        /** Primary only flag. */
+        private boolean primaryOnly;
+
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
 
         /**
+         * Empty constructor for serialization.
          */
-        public RetryFuture(GridCacheContext ctx, TopologyVersionAwareCallable call) {
-            this.ctx = ctx;
-            this.call = call;
-
-            init();
+        public GlobalSizeCallable() {
+            // No-op.
         }
 
         /**
-         * Init.
+         * @param cacheName Cache name.
+         * @param primaryOnly Primary only flag.
          */
-        private void init() {
-            Collection<ClusterNode> nodes = call.nodes(ctx);
-
-            call.topologyVersion(ctx.affinity().affinityTopologyVersion());
-
-            IgniteInternalFuture<Collection<Object>> fut = ctx.closures().callAsyncNoFailover(BROADCAST,
-                F.asSet((Callable<Object>)call), nodes, true);
-
-            fut.listen(new IgniteInClosure<IgniteInternalFuture<Collection<Object>>>() {
-                @Override public void apply(IgniteInternalFuture<Collection<Object>> fut) {
-                    try {
-                        Collection<Object> res = fut.get();
-
-                        onInit();
-
-                        for (Object locRes : res) {
-                            if (locRes == FAIL) {
-                                if (retries-- > 0)
-                                    init();
-                                else {
-                                    onDone(new ClusterTopologyException("Failed to wait topology."));
-
-                                    return;
-                                }
-                            }
+        private GlobalSizeCallable(String cacheName, boolean primaryOnly) {
+            this.cacheName = cacheName;
+            this.primaryOnly = primaryOnly;
+        }
 
-                            onLocal(locRes);
-                        }
+        /** {@inheritDoc} */
+        @Override public Integer apply(Object o) {
+            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
-                        allDone();
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (X.hasCause(e, ClusterTopologyException.class)) {
-                            if (retries-- > 0)
-                                init();
-                            else
-                                onDone(e);
-                        }
-                        else
-                            onDone(e);
-                    }
-                }
-            });
+            return primaryOnly ? cache.primarySize() : cache.size();
         }
 
-        /**
-         * Init reducer.
-         */
-        protected abstract void onInit();
-
-        /**
-         * @param localRes Add local result to global result.
-         */
-        protected abstract void onLocal(Object localRes);
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, cacheName);
+            out.writeBoolean(primaryOnly);
+        }
 
-        /**
-         * On done.
-         */
-        protected abstract void allDone();
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            cacheName = U.readString(in);
+            primaryOnly = in.readBoolean();
+        }
     }
 
     /**
@@ -5665,89 +5697,4 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             metrics.addPutAndGetTimeNanos(System.nanoTime() - start);
         }
     }
-
-    /**
-     * Delayed callable class.
-     */
-    protected static abstract class TopologyVersionAwareCallable<K, V> implements Serializable, Callable<Object> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        protected Ignite ignite;
-
-        /** Affinity topology version. */
-        protected AffinityTopologyVersion topVer;
-
-        /** Cache name. */
-        protected String cacheName;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public TopologyVersionAwareCallable() {
-            // No-op.
-        }
-
-        /**
-         * @param topVer Affinity topology version.
-         */
-        public TopologyVersionAwareCallable(String cacheName, AffinityTopologyVersion topVer) {
-            this.cacheName = cacheName;
-            this.topVer = topVer;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
-            if (!compareTopologyVersions())
-                return FAIL;
-
-            Object res = callLocal();
-
-            if (!compareTopologyVersions())
-                return FAIL;
-            else
-                return res;
-        }
-
-        /**
-         * Call local.
-         *
-         * @return Local result.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected abstract Object callLocal() throws IgniteCheckedException;
-
-        /**
-         * @param ctx Grid cache context.
-         * @return Nodes to call.
-         */
-        protected abstract Collection<ClusterNode> nodes(GridCacheContext ctx);
-
-        /**
-         * Compare topology versions.
-         */
-        public boolean compareTopologyVersions() {
-            GridCacheProcessor cacheProc = ((IgniteKernal) ignite).context().cache();
-
-            GridCacheAdapter<K, V> cacheAdapter = cacheProc.internalCache(cacheName);
-
-            if (cacheAdapter == null)
-                return false;
-
-            final GridCacheContext<K, V> ctx = cacheAdapter.context();
-
-            AffinityTopologyVersion locTopVer = ctx.affinity().affinityTopologyVersion();
-
-            return locTopVer.compareTo(topVer) == 0;
-        }
-
-        /**
-         * @param topVer Affinity topology version.
-         */
-        public void topologyVersion(AffinityTopologyVersion topVer) {
-            this.topVer = topVer;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4bc9297/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 77fa104..c0026ab 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
@@ -124,9 +124,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Must use JDK marshaller since it is used by discovery to fire custom events. */
     private Marshaller marshaller = new JdkMarshaller();
 
-    /** Count down latch for caches. */
-    private CountDownLatch cacheStartedLatch = new CountDownLatch(1);
-
     /**
      * @param ctx Kernal context.
      */
@@ -660,92 +657,87 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        try {
-            if (ctx.config().isDaemon())
-                return;
+        if (ctx.config().isDaemon())
+            return;
 
-            ClusterNode locNode = ctx.discovery().localNode();
+        ClusterNode locNode = ctx.discovery().localNode();
 
-            // Init cache plugin managers.
-            final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
+        // Init cache plugin managers.
+        final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
 
-            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                CacheConfiguration locCcfg = desc.cacheConfiguration();
+        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+            CacheConfiguration locCcfg = desc.cacheConfiguration();
 
-                CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
+            CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
 
-                cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
-            }
+            cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
+        }
 
-            if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
-                for (ClusterNode n : ctx.discovery().remoteNodes()) {
-                    checkTransactionConfiguration(n);
+        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
+            for (ClusterNode n : ctx.discovery().remoteNodes()) {
+                checkTransactionConfiguration(n);
 
-                    DeploymentMode locDepMode = ctx.config().getDeploymentMode();
-                    DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
+                DeploymentMode locDepMode = ctx.config().getDeploymentMode();
+                DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
 
-                    CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
-                        locDepMode, rmtDepMode, true);
+                CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
+                    locDepMode, rmtDepMode, true);
 
-                    for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                        CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
+                for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                    CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
 
-                        if (rmtCfg != null) {
-                            CacheConfiguration locCfg = desc.cacheConfiguration();
+                    if (rmtCfg != null) {
+                        CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                            checkCache(locCfg, rmtCfg, n);
+                        checkCache(locCfg, rmtCfg, n);
 
-                            // Check plugin cache configurations.
-                            CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
+                        // Check plugin cache configurations.
+                        CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
 
-                            assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                        assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                            pluginMgr.validateRemotes(rmtCfg, n);
-                        }
+                        pluginMgr.validateRemotes(rmtCfg, n);
                     }
                 }
             }
+        }
 
-            // Start dynamic caches received from collect discovery data.
-            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                boolean started = desc.onStart();
+        // Start dynamic caches received from collect discovery data.
+        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+            boolean started = desc.onStart();
 
-                assert started : "Failed to change started flag for locally configured cache: " + desc;
+            assert started : "Failed to change started flag for locally configured cache: " + desc;
 
-                desc.clearRemoteConfigurations();
+            desc.clearRemoteConfigurations();
 
-                CacheConfiguration ccfg = desc.cacheConfiguration();
+            CacheConfiguration ccfg = desc.cacheConfiguration();
 
-                IgnitePredicate filter = ccfg.getNodeFilter();
+            IgnitePredicate filter = ccfg.getNodeFilter();
 
-                if (filter.apply(locNode)) {
-                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+            if (filter.apply(locNode)) {
+                CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-                    CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
+                CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
 
-                    assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                    GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
+                GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
 
-                    ctx.dynamicDeploymentId(desc.deploymentId());
+                ctx.dynamicDeploymentId(desc.deploymentId());
 
-                    sharedCtx.addCacheContext(ctx);
+                sharedCtx.addCacheContext(ctx);
 
-                    GridCacheAdapter cache = ctx.cache();
+                GridCacheAdapter cache = ctx.cache();
 
-                    String name = ccfg.getName();
+                String name = ccfg.getName();
 
-                    caches.put(maskNull(name), cache);
+                caches.put(maskNull(name), cache);
 
-                    startCache(cache);
+                startCache(cache);
 
-                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
-                }
+                jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
             }
         }
-        finally {
-            cacheStartedLatch.countDown();
-        }
 
         ctx.marshallerContext().onMarshallerCacheStarted(ctx);
 
@@ -843,8 +835,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStop(boolean cancel) {
-        cacheStartedLatch.countDown();
-
         if (ctx.config().isDaemon())
             return;
 
@@ -2696,13 +2686,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting internal cache adapter: " + name);
 
-        try {
-            cacheStartedLatch.await();
-        }
-        catch (InterruptedException e) {
-            throw new IgniteException("Failed to wait starting caches.");
-        }
-
         return (GridCacheAdapter<K, V>)caches.get(maskNull(name));
     }
 


[42/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the withAsync method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the withAsync method.


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

Branch: refs/heads/ignite-37
Commit: 281f4ef208e7928c8514992458f34ab96781f671
Parents: 2cb7317
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:50:18 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:50:18 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/281f4ef2/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
index f5b5153..be05a38 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
@@ -24,7 +24,7 @@ public interface IgniteAsyncSupport {
     /**
      * Gets instance of this component with asynchronous mode enabled.
      *
-     * @return Component with asynchronous mode enabled.
+     * @return Instance of this component with asynchronous mode enabled.
      */
     public IgniteAsyncSupport withAsync();
 


[32/50] incubator-ignite git commit: #ignite-732: IgniteCache.size() should not fail in case of topology changes.

Posted by sb...@apache.org.
#ignite-732: IgniteCache.size() should not fail in case of topology changes.


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

Branch: refs/heads/ignite-37
Commit: 139aa270ae61494c0757867f2dc531ec7251b1da
Parents: 0885ac0
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Apr 30 18:43:56 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Apr 30 18:43:56 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 ++++++++++---------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 3 files changed, 349 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/139aa270/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
index c2ad198..87081fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
@@ -24,15 +24,16 @@ import java.util.*;
 
 /**
  * Convenience adapter for {@link ComputeTask} interface. Here is an example of
- * how {@code GridComputeTaskAdapter} can be used:
+ * how {@code ComputeTaskAdapter} can be used:
  * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ * public class MyFooBarTask extends ComputeTaskAdapter&lt;String, String&gt; {
  *     // Inject load balancer.
  *     &#64;LoadBalancerResource
  *     ComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg)
+ *         throws IgniteCheckedException {
  *         Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -76,8 +77,8 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * <p>
      * If remote job resulted in exception ({@link ComputeJobResult#getException()} is not {@code null}),
      * then {@link ComputeJobResultPolicy#FAILOVER} policy will be returned if the exception is instance
-     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException}, which means that
-     * remote node either failed or job execution was rejected before it got a chance to start. In all
+     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException},
+     * which means that remote node either failed or job execution was rejected before it got a chance to start. In all
      * other cases the exception will be rethrown which will ultimately cause task to fail.
      *
      * @param res Received remote grid executable result.
@@ -87,7 +88,8 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * @throws IgniteException If handling a job result caused an error effectively rejecting
      *      a failover. This exception will be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd)
+        throws IgniteException {
         IgniteException e = res.getException();
 
         // Try to failover if result is failed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/139aa270/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 3f4e97b..39f19b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -75,6 +75,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Failed result. */
+    private static final Object FAIL = new Integer(-1);
+
     /** clearLocally() split threshold. */
     public static final int CLEAR_ALL_SPLIT_THRESHOLD = 10000;
 
@@ -882,7 +885,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<Cache.Entry<K, V>> entrySet() {
-        return entrySet((CacheEntryPredicate[]) null);
+        return entrySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
@@ -897,17 +900,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> keySet() {
-        return keySet((CacheEntryPredicate[]) null);
+        return keySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
-        return primaryKeySet((CacheEntryPredicate[]) null);
+        return primaryKeySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<V> values() {
-        return values((CacheEntryPredicate[]) null);
+        return values((CacheEntryPredicate[])null);
     }
 
     /**
@@ -1080,36 +1083,31 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void clear() throws IgniteCheckedException {
-        // Clear local cache synchronously.
-        clearLocally();
-
-        clearRemotes(0, new GlobalClearAllCallable(name()));
+        clearAll(0, new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
     }
 
     /** {@inheritDoc} */
     @Override public void clear(K key) throws IgniteCheckedException {
-        // Clear local cache synchronously.
-        clearLocally(key);
-
-        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
+        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public void clearAll(Set<? extends K> keys) throws IgniteCheckedException {
-        // Clear local cache synchronously.
-        clearLocallyAll(keys);
-
-        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), keys));
+        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            keys));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(K key) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(Set<? extends K> keys) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), keys));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            keys));
     }
 
     /**
@@ -1118,19 +1116,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param clearCall Global clear callable object.
      * @throws IgniteCheckedException In case of cache could not be cleared on any of the nodes.
      */
-    private void clearRemotes(long timeout, GlobalClearCallable clearCall) throws IgniteCheckedException {
+    private void clearAll(long timeout, TopologyVersionAwareCallable clearCall) throws IgniteCheckedException {
         try {
-            // Send job to remote nodes only.
-            Collection<ClusterNode> nodes =
-                ctx.grid().cluster().forCacheNodes(name(), true, true, false).forRemotes().nodes();
-
             IgniteInternalFuture<Object> fut = null;
 
-            if (!nodes.isEmpty()) {
-                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
+            ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
 
-                fut = ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
-            }
+            fut = new ClearFuture(ctx, clearCall);
 
             if (fut != null)
                 fut.get();
@@ -1149,19 +1141,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync() {
-        return clearAsync(new GlobalClearAllCallable(name()));
+        return clearAsync(new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
     }
 
     /**
      * @param clearCall Global clear callable object.
      * @return Future.
      */
-    private IgniteInternalFuture<?> clearAsync(GlobalClearCallable clearCall) {
+    private IgniteInternalFuture<?> clearAsync(TopologyVersionAwareCallable clearCall) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name(), true, true, false).nodes();
 
         if (!nodes.isEmpty()) {
-            IgniteInternalFuture<Object> fut =
-                ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
+            IgniteInternalFuture<Object> fut = new ClearFuture(ctx, clearCall);
 
             return fut.chain(new CX1<IgniteInternalFuture<Object>, Object>() {
                 @Override public Object applyx(IgniteInternalFuture<Object> fut) throws IgniteCheckedException {
@@ -2117,7 +2108,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 throws IgniteCheckedException {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = F.viewAsMap(keys,
                     new C1<K, EntryProcessor<K, V, Object>>() {
-                            @Override public EntryProcessor apply(K k) {
+                        @Override public EntryProcessor apply(K k) {
                             return entryProcessor;
                         }
                     });
@@ -2145,7 +2136,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp() {
             @Override public IgniteInternalFuture<GridCacheReturn> inOp(IgniteTxLocalAdapter tx) {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap =
-                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>) entryProcessor);
+                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor);
 
                 return tx.invokeAsync(ctx, invokeMap, args);
             }
@@ -2371,7 +2362,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.noValArray())
-                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>) RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
             }
 
             @Override public String toString() {
@@ -2526,7 +2517,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasValArray()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>) RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2915,7 +2906,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (ctx.deploymentEnabled())
                     ctx.deploy().registerClass(oldVal);
 
-                return (GridCacheReturn) tx.putAllAsync(ctx,
+                return tx.putAllAsync(ctx,
                         F.t(key, newVal),
                         true,
                         null,
@@ -3017,7 +3008,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     ctx.deploy().registerClass(val);
 
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), null, false,
-                        ctx.equalsValArray(val)).get().success();
+                    ctx.equalsValArray(val)).get().success();
             }
 
             @Override public String toString() {
@@ -3230,10 +3221,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         TransactionConfiguration cfg = ctx.gridConfig().getTransactionConfiguration();
 
         return txStart(
-                concurrency,
-                isolation,
-                cfg.getDefaultTxTimeout(),
-                0
+            concurrency,
+            isolation,
+            cfg.getDefaultTxTimeout(),
+            0
         );
     }
 
@@ -3576,22 +3567,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0);
 
-        IgniteInternalFuture<Collection<Integer>> fut =
-            ctx.closures().broadcastNoFailover(new SizeCallable(ctx.name(), peekModes), null, nodes);
-
-        return fut.chain(new CX1<IgniteInternalFuture<Collection<Integer>>, Integer>() {
-            @Override public Integer applyx(IgniteInternalFuture<Collection<Integer>> fut)
-            throws IgniteCheckedException {
-                Collection<Integer> res = fut.get();
-
-                int totalSize = 0;
-
-                for (Integer size : res)
-                    totalSize += size;
-
-                return totalSize;
-            }
-        });
+        return new SizeFuture(peekModes, ctx, modes.near);
     }
 
     /** {@inheritDoc} */
@@ -3675,7 +3651,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return F.iterator(iterator(),
             new IgniteClosure<Cache.Entry<K, V>, Cache.Entry<K, V>>() {
                 private IgniteCacheExpiryPolicy expiryPlc =
-                        ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
+                    ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
 
                 @Override public Cache.Entry<K, V> apply(Cache.Entry<K, V> lazyEntry) {
                     CacheOperationContext prev = ctx.gate().enter(opCtx);
@@ -3909,50 +3885,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Gets cache global size (with or without backups).
-     *
-     * @param primaryOnly {@code True} if only primary sizes should be included.
-     * @return Global size.
-     * @throws IgniteCheckedException If internal task execution failed.
-     */
-    private int globalSize(boolean primaryOnly) throws IgniteCheckedException {
-        try {
-            // Send job to remote nodes only.
-            Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name()).forRemotes().nodes();
-
-            IgniteInternalFuture<Collection<Integer>> fut = null;
-
-            if (!nodes.isEmpty()) {
-                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, gridCfg.getNetworkTimeout());
-
-                fut = ctx.closures().broadcastNoFailover(new GlobalSizeCallable(name(), primaryOnly), null, nodes);
-            }
-
-            // Get local value.
-            int globalSize = primaryOnly ? primarySize() : size();
-
-            if (fut != null) {
-                for (Integer i : fut.get())
-                    globalSize += i;
-            }
-
-            return globalSize;
-        }
-        catch (ClusterGroupEmptyCheckedException ignore) {
-            if (log.isDebugEnabled())
-                log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
-
-            return primaryOnly ? primarySize() : size();
-        }
-        catch (ComputeTaskTimeoutCheckedException e) {
-            U.warn(log, "Timed out waiting for remote nodes to finish cache clear (consider increasing " +
-                "'networkTimeout' configuration property) [cacheName=" + name() + "]");
-
-            throw e;
-        }
-    }
-
-    /**
      * @param op Cache operation.
      * @param <T> Return type.
      * @return Operation result.
@@ -4893,47 +4825,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Internal callable which performs clear operation on a cache with the given name.
-     */
-    @GridInternal
-    private static abstract class GlobalClearCallable implements Callable<Object>, Externalizable {
-        /** Cache name. */
-        protected String cacheName;
-
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        protected Ignite ignite;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public GlobalClearCallable() {
-            // No-op.
-        }
-
-        /**
-         * @param cacheName Cache name.
-         */
-        protected GlobalClearCallable(String cacheName) {
-            this.cacheName = cacheName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
-        }
-    }
-
-    /**
      * Global clear all.
      */
     @GridInternal
-    private static class GlobalClearAllCallable extends GlobalClearCallable {
+    private static class GlobalClearAllCallable extends TopologyVersionAwareCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4946,24 +4841,30 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          */
-        private GlobalClearAllCallable(String cacheName) {
-            super(cacheName);
+        private GlobalClearAllCallable(String cacheName, AffinityTopologyVersion topVer) {
+            super(cacheName, topVer);
         }
 
         /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
+        @Override protected Object callLocal() {
             ((IgniteEx)ignite).cachex(cacheName).clearLocally();
 
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
+            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
+        }
     }
 
     /**
      * Global clear keys.
      */
     @GridInternal
-    private static class GlobalClearKeySetCallable<K, V> extends GlobalClearCallable {
+    private static class GlobalClearKeySetCallable<K, V> extends TopologyVersionAwareCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4979,33 +4880,25 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          * @param keys Keys to clear.
          */
-        private GlobalClearKeySetCallable(String cacheName, Set<? extends K> keys) {
-            super(cacheName);
+        protected GlobalClearKeySetCallable(String cacheName, AffinityTopologyVersion topVer, Set<? extends K> keys) {
+            super(cacheName, topVer);
 
             this.keys = keys;
         }
 
         /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
-            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
-
-            return null;
+        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
+            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
         }
 
         /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
-            out.writeObject(keys);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
+        @Override protected Object callLocal() {
+            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
 
-            keys = (Set<K>) in.readObject();
+            return null;
         }
     }
 
@@ -5013,127 +4906,202 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Internal callable for global size calculation.
      */
     @GridInternal
-    private static class SizeCallable extends IgniteClosureX<Object, Integer> implements Externalizable {
+    private static class GlobalSizeCallable extends TopologyVersionAwareCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
-        /** Cache name. */
-        private String cacheName;
-
         /** Peek modes. */
         private CachePeekMode[] peekModes;
 
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        private Ignite ignite;
+        /** Near enable. */
+        private boolean nearEnable;
 
         /**
          * Required by {@link Externalizable}.
          */
-        public SizeCallable() {
+        public GlobalSizeCallable() {
             // No-op.
         }
 
         /**
          * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          * @param peekModes Cache peek modes.
          */
-        private SizeCallable(String cacheName, CachePeekMode[] peekModes) {
-            this.cacheName = cacheName;
+        private GlobalSizeCallable(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes, boolean nearEnable) {
+            super(cacheName, topVer);
+
             this.peekModes = peekModes;
+            this.nearEnable = nearEnable;
         }
 
         /** {@inheritDoc} */
-        @Override public Integer applyx(Object o) throws IgniteCheckedException {
-            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
-
-            assert cache != null : cacheName;
+        @Override protected Object callLocal() {
+            try {
+                IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
-            return cache.localSize(peekModes);
+                return cache == null ? 0 : cache.localSize(peekModes);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
         }
 
         /** {@inheritDoc} */
-        @SuppressWarnings("ForLoopReplaceableByForEach")
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
+        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
+            IgniteClusterEx cluster = ctx.grid().cluster();
 
-            out.writeInt(peekModes.length);
+            ClusterGroup grp = nearEnable ? cluster.forCacheNodes(ctx.name(), true, true, false) : cluster.forDataNodes(ctx.name());
 
-            for (int i = 0; i < peekModes.length; i++)
-                U.writeEnum(out, peekModes[i]);
+            return grp.nodes();
         }
 
         /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
+        public String toString() {
+            return S.toString(GlobalSizeCallable.class, this);
+        }
+    }
 
-            int len = in.readInt();
+    /**
+     * Cache size future.
+     */
+    private static class SizeFuture extends RetryFuture {
+        /** Size. */
+        private int size = 0;
 
-            peekModes = new CachePeekMode[len];
+        /**
+         * @param peekModes Peek modes.
+         */
+        public SizeFuture(CachePeekMode[] peekModes, GridCacheContext ctx, boolean near) {
+            super(ctx, new GlobalSizeCallable(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes, near));
+        }
 
-            for (int i = 0; i < len; i++)
-                peekModes[i] = CachePeekMode.fromOrdinal(in.readByte());
+        /** {@inheritDoc} */
+        @Override protected void onInit() {
+            size = 0;
         }
 
         /** {@inheritDoc} */
-        public String toString() {
-            return S.toString(SizeCallable.class, this);
+        @Override protected void onLocal(Object localRes) {
+            size += (Integer)localRes;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void allDone() {
+            onDone(size);
         }
     }
 
     /**
-     * Internal callable which performs {@link IgniteInternalCache#size()} or {@link IgniteInternalCache#primarySize()}
-     * operation on a cache with the given name.
+     * Cache clear future.
      */
-    @GridInternal
-    private static class GlobalSizeCallable implements IgniteClosure<Object, Integer>, Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
+    private static class ClearFuture extends RetryFuture {
+        /**
+         */
+        public ClearFuture(GridCacheContext ctx, TopologyVersionAwareCallable clearCall) {
+            super(ctx, clearCall);
+        }
 
-        /** Cache name. */
-        private String cacheName;
+        /** {@inheritDoc} */
+        @Override protected void onInit() {
+           // No-op.
+        }
 
-        /** Primary only flag. */
-        private boolean primaryOnly;
+        /** {@inheritDoc} */
+        @Override protected void onLocal(Object localRes) {
+            // No-op.
+        }
 
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        private Ignite ignite;
+        /** {@inheritDoc} */
+        @Override protected void allDone() {
+            onDone();
+        }
+    }
+
+    /**
+     * Retry future.
+     */
+    protected static abstract class RetryFuture<T> extends GridFutureAdapter<T> {
+        /** Context. */
+        private final GridCacheContext ctx;
+
+        /** Callable. */
+        private final TopologyVersionAwareCallable call;
+
+        /** Max retries count before issuing an error. */
+        private volatile int retries = 32;
 
         /**
-         * Empty constructor for serialization.
          */
-        public GlobalSizeCallable() {
-            // No-op.
+        public RetryFuture(GridCacheContext ctx, TopologyVersionAwareCallable call) {
+            this.ctx = ctx;
+            this.call = call;
+
+            init();
         }
 
         /**
-         * @param cacheName Cache name.
-         * @param primaryOnly Primary only flag.
+         * Init.
          */
-        private GlobalSizeCallable(String cacheName, boolean primaryOnly) {
-            this.cacheName = cacheName;
-            this.primaryOnly = primaryOnly;
-        }
+        private void init() {
+            Collection<ClusterNode> nodes = call.nodes(ctx);
 
-        /** {@inheritDoc} */
-        @Override public Integer apply(Object o) {
-            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
+            call.topologyVersion(ctx.affinity().affinityTopologyVersion());
 
-            return primaryOnly ? cache.primarySize() : cache.size();
-        }
+            IgniteInternalFuture<Collection<Object>> fut = ctx.closures().callAsyncNoFailover(BROADCAST,
+                F.asSet((Callable<Object>)call), nodes, true);
 
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
-            out.writeBoolean(primaryOnly);
-        }
+            fut.listen(new IgniteInClosure<IgniteInternalFuture<Collection<Object>>>() {
+                @Override public void apply(IgniteInternalFuture<Collection<Object>> fut) {
+                    try {
+                        Collection<Object> res = fut.get();
 
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
-            primaryOnly = in.readBoolean();
+                        onInit();
+
+                        for (Object locRes : res) {
+                            if (locRes == FAIL) {
+                                if (retries-- > 0)
+                                    init();
+                                else {
+                                    onDone(new ClusterTopologyException("Failed to wait topology."));
+
+                                    return;
+                                }
+                            }
+
+                            onLocal(locRes);
+                        }
+
+                        allDone();
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (X.hasCause(e, ClusterTopologyException.class)) {
+                            if (retries-- > 0)
+                                init();
+                            else
+                                onDone(e);
+                        }
+                        else
+                            onDone(e);
+                    }
+                }
+            });
         }
+
+        /**
+         * Init reducer.
+         */
+        protected abstract void onInit();
+
+        /**
+         * @param localRes Add local result to global result.
+         */
+        protected abstract void onLocal(Object localRes);
+
+        /**
+         * On done.
+         */
+        protected abstract void allDone();
     }
 
     /**
@@ -5697,4 +5665,89 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             metrics.addPutAndGetTimeNanos(System.nanoTime() - start);
         }
     }
+
+    /**
+     * Delayed callable class.
+     */
+    protected static abstract class TopologyVersionAwareCallable<K, V> implements Serializable, Callable<Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        protected Ignite ignite;
+
+        /** Affinity topology version. */
+        protected AffinityTopologyVersion topVer;
+
+        /** Cache name. */
+        protected String cacheName;
+
+        /**
+         * Empty constructor for serialization.
+         */
+        public TopologyVersionAwareCallable() {
+            // No-op.
+        }
+
+        /**
+         * @param topVer Affinity topology version.
+         */
+        public TopologyVersionAwareCallable(String cacheName, AffinityTopologyVersion topVer) {
+            this.cacheName = cacheName;
+            this.topVer = topVer;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            if (!compareTopologyVersions())
+                return FAIL;
+
+            Object res = callLocal();
+
+            if (!compareTopologyVersions())
+                return FAIL;
+            else
+                return res;
+        }
+
+        /**
+         * Call local.
+         *
+         * @return Local result.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected abstract Object callLocal() throws IgniteCheckedException;
+
+        /**
+         * @param ctx Grid cache context.
+         * @return Nodes to call.
+         */
+        protected abstract Collection<ClusterNode> nodes(GridCacheContext ctx);
+
+        /**
+         * Compare topology versions.
+         */
+        public boolean compareTopologyVersions() {
+            GridCacheProcessor cacheProc = ((IgniteKernal) ignite).context().cache();
+
+            GridCacheAdapter<K, V> cacheAdapter = cacheProc.internalCache(cacheName);
+
+            if (cacheAdapter == null)
+                return false;
+
+            final GridCacheContext<K, V> ctx = cacheAdapter.context();
+
+            AffinityTopologyVersion locTopVer = ctx.affinity().affinityTopologyVersion();
+
+            return locTopVer.compareTo(topVer) == 0;
+        }
+
+        /**
+         * @param topVer Affinity topology version.
+         */
+        public void topologyVersion(AffinityTopologyVersion topVer) {
+            this.topVer = topVer;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/139aa270/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 c0026ab..77fa104 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
@@ -124,6 +124,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Must use JDK marshaller since it is used by discovery to fire custom events. */
     private Marshaller marshaller = new JdkMarshaller();
 
+    /** Count down latch for caches. */
+    private CountDownLatch cacheStartedLatch = new CountDownLatch(1);
+
     /**
      * @param ctx Kernal context.
      */
@@ -657,87 +660,92 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
-            return;
+        try {
+            if (ctx.config().isDaemon())
+                return;
 
-        ClusterNode locNode = ctx.discovery().localNode();
+            ClusterNode locNode = ctx.discovery().localNode();
 
-        // Init cache plugin managers.
-        final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
+            // Init cache plugin managers.
+            final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
 
-        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-            CacheConfiguration locCcfg = desc.cacheConfiguration();
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                CacheConfiguration locCcfg = desc.cacheConfiguration();
 
-            CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
+                CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
 
-            cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
-        }
+                cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
+            }
 
-        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
-            for (ClusterNode n : ctx.discovery().remoteNodes()) {
-                checkTransactionConfiguration(n);
+            if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
+                for (ClusterNode n : ctx.discovery().remoteNodes()) {
+                    checkTransactionConfiguration(n);
 
-                DeploymentMode locDepMode = ctx.config().getDeploymentMode();
-                DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
+                    DeploymentMode locDepMode = ctx.config().getDeploymentMode();
+                    DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
 
-                CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
-                    locDepMode, rmtDepMode, true);
+                    CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
+                        locDepMode, rmtDepMode, true);
 
-                for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                    CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
+                    for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                        CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
 
-                    if (rmtCfg != null) {
-                        CacheConfiguration locCfg = desc.cacheConfiguration();
+                        if (rmtCfg != null) {
+                            CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                        checkCache(locCfg, rmtCfg, n);
+                            checkCache(locCfg, rmtCfg, n);
 
-                        // Check plugin cache configurations.
-                        CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
+                            // Check plugin cache configurations.
+                            CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
 
-                        assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                            assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                        pluginMgr.validateRemotes(rmtCfg, n);
+                            pluginMgr.validateRemotes(rmtCfg, n);
+                        }
                     }
                 }
             }
-        }
 
-        // Start dynamic caches received from collect discovery data.
-        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-            boolean started = desc.onStart();
+            // Start dynamic caches received from collect discovery data.
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                boolean started = desc.onStart();
 
-            assert started : "Failed to change started flag for locally configured cache: " + desc;
+                assert started : "Failed to change started flag for locally configured cache: " + desc;
 
-            desc.clearRemoteConfigurations();
+                desc.clearRemoteConfigurations();
 
-            CacheConfiguration ccfg = desc.cacheConfiguration();
+                CacheConfiguration ccfg = desc.cacheConfiguration();
 
-            IgnitePredicate filter = ccfg.getNodeFilter();
+                IgnitePredicate filter = ccfg.getNodeFilter();
 
-            if (filter.apply(locNode)) {
-                CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+                if (filter.apply(locNode)) {
+                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-                CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
+                    CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
 
-                assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                    assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
+                    GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
 
-                ctx.dynamicDeploymentId(desc.deploymentId());
+                    ctx.dynamicDeploymentId(desc.deploymentId());
 
-                sharedCtx.addCacheContext(ctx);
+                    sharedCtx.addCacheContext(ctx);
 
-                GridCacheAdapter cache = ctx.cache();
+                    GridCacheAdapter cache = ctx.cache();
 
-                String name = ccfg.getName();
+                    String name = ccfg.getName();
 
-                caches.put(maskNull(name), cache);
+                    caches.put(maskNull(name), cache);
 
-                startCache(cache);
+                    startCache(cache);
 
-                jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
+                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
+                }
             }
         }
+        finally {
+            cacheStartedLatch.countDown();
+        }
 
         ctx.marshallerContext().onMarshallerCacheStarted(ctx);
 
@@ -835,6 +843,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStop(boolean cancel) {
+        cacheStartedLatch.countDown();
+
         if (ctx.config().isDaemon())
             return;
 
@@ -2686,6 +2696,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting internal cache adapter: " + name);
 
+        try {
+            cacheStartedLatch.await();
+        }
+        catch (InterruptedException e) {
+            throw new IgniteException("Failed to wait starting caches.");
+        }
+
         return (GridCacheAdapter<K, V>)caches.get(maskNull(name));
     }
 


[06/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: a55477970a2fdae5e93bcace708c98892937b97b
Parents: a8a9d10
Author: avinogradov <av...@gridgain.com>
Authored: Mon Apr 27 17:42:03 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Mon Apr 27 17:42:03 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java | 6 +++---
 .../cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java   | 2 +-
 .../cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java       | 2 +-
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5547797/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index b358b15..2a3f08a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -158,7 +158,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         failAtomicPut(++key);
 
         //Check that cache is empty.
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         assert jcache(0).get(new TestKey(String.valueOf(key))) == null;
 
@@ -168,7 +168,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         failAtomicPut(++key);
 
         //Check that cache is not empty.
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         assert jcache(0).get(new TestKey(String.valueOf(key))) != null;
 
@@ -178,7 +178,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         failAtomicGet(++key);
 
         //GridNearGetResponse unmarshalling failed test
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         jcache(0).put(new TestKey(String.valueOf(++key)), "");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5547797/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
index f247a00..95d38e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -31,7 +31,7 @@ public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() throws Exception {
         //GridDhtPartitionSupplyMessage unmarshalling failed test
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         for (int i = 0; i <= 20; i++) {
             jcache(0).put(new TestKey(String.valueOf(++key)), "");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a5547797/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index 265490c..afb96be 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -37,7 +37,7 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
 
     /** {@inheritDoc} */
     @Override public void testResponseMessageOnUnmarshallingFailed() {
-        readCnt.set(100);
+        readCnt.set(Integer.MAX_VALUE);
 
         TestKey val = new TestKey(String.valueOf(++key));
 


[46/50] incubator-ignite git commit: # ignite-sprint-5 fixed test

Posted by sb...@apache.org.
# ignite-sprint-5 fixed 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/da3d9e73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/da3d9e73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/da3d9e73

Branch: refs/heads/ignite-37
Commit: da3d9e73d182b77d27902871c19817f439308b2b
Parents: 114a852
Author: sboikov <sb...@gridgain.com>
Authored: Tue May 5 09:36:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue May 5 09:36:52 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridCacheDhtPreloadSelfTest.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/da3d9e73/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index f04b87f..5b63c6a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -266,7 +266,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
                         ClusterNode node = evt.discoveryNode();
 
                         return evt.type() == EVT_CACHE_REBALANCE_STOPPED && node.id().equals(nodeId) &&
-                            evt.discoveryEventType() == EVT_NODE_LEFT;
+                            (evt.discoveryEventType() == EVT_NODE_LEFT || evt.discoveryEventType() == EVT_NODE_FAILED);
                     }
                 }, EVT_CACHE_REBALANCE_STOPPED));
 


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

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


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

Branch: refs/heads/ignite-37
Commit: d59e4c42fbf3a55afbba9dae3d9e81c9f13e98fa
Parents: 8ae6b03 47136b5
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Apr 29 18:01:14 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Apr 29 18:01:14 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++
 modules/cloud/pom.xml                           | 106 ++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 +++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 ++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../configuration/CacheConfiguration.java       |   4 +-
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |  28 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../processors/cache/IgniteCacheProxy.java      |  10 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 ++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 ++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../visor/cache/VisorCacheStartTask.java        | 155 ++++++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |  12 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../distributed/GridCacheLockAbstractTest.java  |  75 +++
 .../DataStreamerMultiThreadedSelfTest.java      |   2 +
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++
 modules/gce/pom.xml                             |  92 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 +++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 +++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 +++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 ++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 +++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 ++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 ++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 +++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 ++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 +++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 ++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 ++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 +++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 ++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 ++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 +++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 +++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 ++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  36 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 439 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 275 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 182 files changed, 8304 insertions(+), 3154 deletions(-)
----------------------------------------------------------------------



[39/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the close method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the close method.


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

Branch: refs/heads/ignite-37
Commit: f298d7216a81b0416c1cf123efd99fcff2d41e19
Parents: 09a33fe
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:28:44 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:28:44 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/examples/streaming/wordcount/CacheConfig.java | 5 -----
 modules/core/src/main/java/org/apache/ignite/IgniteCache.java   | 5 +++++
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f298d721/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java b/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
index bb2a18e..58704ca 100644
--- a/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
+++ b/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.examples.streaming.wordcount;
 
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.configuration.*;
 
 import javax.cache.configuration.*;
@@ -44,10 +43,6 @@ public class CacheConfig {
         // Sliding window of 1 seconds.
         cfg.setExpiryPolicyFactory(FactoryBuilder.factoryOf(new CreatedExpiryPolicy(new Duration(SECONDS, 1))));
 
-        // Do not allow more than 1 million entries.
-        // Allows to run this example with smaller available memory.
-        cfg.setEvictionPolicy(new FifoEvictionPolicy<>(1_000_000));
-
         return cfg;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f298d721/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index d99902a..bf4080a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -499,6 +499,11 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
         CacheEntryProcessor<K, V, T> entryProcessor, Object... args);
 
     /**
+     * Completely deletes the cache with all its data from the system.
+     */
+    @Override void close();
+
+    /**
      * This cache node to re-balance its partitions. This method is usually used when
      * {@link CacheConfiguration#getRebalanceDelay()} configuration parameter has non-zero value.
      * When many nodes are started or stopped almost concurrently, it is more efficient to delay


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

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-sprint-5


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

Branch: refs/heads/ignite-37
Commit: d0798504652d09b583c7e9436dd619461aea1b05
Parents: b4c424f 7121619
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 16:12:38 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 16:12:38 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 168 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 pom.xml                                         |  51 ++++++
 4 files changed, 166 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


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


[17/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: b0374bca8929687f6398076c795b68673fa244ca
Parents: b45679c
Author: avinogradov <av...@gridgain.com>
Authored: Wed Apr 29 15:25:22 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Apr 29 15:25:22 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 13 +++++++
 .../dht/preloader/GridDhtForceKeysResponse.java | 36 ++++++++++++++++----
 2 files changed, 42 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0374bca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 14d6f7c..a6cbc37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -302,6 +302,13 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
+    /**
+     * Sends response on failed message.
+     * @param nodeId node id.
+     * @param res response.
+     * @param cctx shared context.
+     * @param plc grid io policy.
+     */
     private void sendResponseOnFailedMessage(UUID nodeId, GridCacheMessage res, GridCacheSharedContext cctx,
         GridIoPolicy plc) {
         try {
@@ -313,6 +320,12 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
         }
     }
 
+    /**
+     * Processes failed messages.
+     * @param nodeId niode id.
+     * @param msg message.
+     * @throws IgniteCheckedException
+     */
     private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0374bca/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 5c6e24f..41ce0be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -43,8 +43,12 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
     private IgniteUuid miniId;
 
     /** Error. */
+    @GridDirectTransient
     private volatile IgniteCheckedException err;
 
+    /** Serialized error. */
+    private byte[] errBytes;
+
     /** Missed (not found) keys. */
     @GridToStringInclude
     @GridDirectCollection(KeyCacheObject.class)
@@ -160,6 +164,8 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
             for (GridCacheEntryInfo info : infos)
                 info.marshal(cctx);
         }
+
+        errBytes = ctx.marshaller().marshal(err);
     }
 
     /** {@inheritDoc} */
@@ -175,6 +181,8 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
             for (GridCacheEntryInfo info : infos)
                 info.unmarshal(cctx, ldr);
         }
+
+        err = ctx.marshaller().unmarshal(errBytes, ldr);
     }
 
     /** {@inheritDoc} */
@@ -193,24 +201,30 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
 
         switch (writer.state()) {
             case 3:
-                if (!writer.writeIgniteUuid("futId", futId))
+                if (!writer.writeByteArray("errBytes", errBytes))
                     return false;
 
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeCollection("infos", infos, MessageCollectionItemType.MSG))
+                if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeCollection("infos", infos, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 6:
+                if (!writer.writeIgniteUuid("miniId", miniId))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
                 if (!writer.writeCollection("missedKeys", missedKeys, MessageCollectionItemType.MSG))
                     return false;
 
@@ -233,7 +247,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
 
         switch (reader.state()) {
             case 3:
-                futId = reader.readIgniteUuid("futId");
+                errBytes = reader.readByteArray("errBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -241,7 +255,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
                 reader.incrementState();
 
             case 4:
-                infos = reader.readCollection("infos", MessageCollectionItemType.MSG);
+                futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -249,7 +263,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
                 reader.incrementState();
 
             case 5:
-                miniId = reader.readIgniteUuid("miniId");
+                infos = reader.readCollection("infos", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -257,6 +271,14 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
                 reader.incrementState();
 
             case 6:
+                miniId = reader.readIgniteUuid("miniId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
                 missedKeys = reader.readCollection("missedKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -276,7 +298,7 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 7;
+        return 8;
     }
 
     /** {@inheritDoc} */


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

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


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

Branch: refs/heads/ignite-37
Commit: 44b00ea46975ca8894ffbb24d53947ff4571d075
Parents: 04369fe 139aa27
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:51:39 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:51:39 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 ++++++++++---------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 3 files changed, 349 insertions(+), 277 deletions(-)
----------------------------------------------------------------------



[03/50] incubator-ignite git commit: ignite-646

Posted by sb...@apache.org.
ignite-646


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

Branch: refs/heads/ignite-37
Commit: 163be30b3b0460967cdba5f8d093eb4ce3c1007a
Parents: db7aa24
Author: avinogradov <av...@gridgain.com>
Authored: Fri Apr 24 19:42:49 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Fri Apr 24 19:42:49 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheP2pUnmarshallingErrorTest.java          | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/163be30b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index a50f07c..277f10d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -32,7 +32,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     /** Allows to change behavior of readExternal method. */
     protected static AtomicInteger readCnt = new AtomicInteger();
 
-    /** iterable key */
+    /** Iterable key. */
     protected static int key = 0;
 
     /** {@inheritDoc} */
@@ -81,7 +81,7 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         public TestKey() {
         }
 
-        /** field. */
+        /** Field. */
         private String field;
 
         /** {@inheritDoc} */
@@ -160,15 +160,17 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
         failAtomicGet();
 
+        //Check that cache is empty.
         readCnt.set(100);
 
         assert jcache(0).get(new TestKey("1")) == null;
 
+        //GridDhtAtomicUpdateRequest unmarshalling failed test
         readCnt.set(2);
 
-        //GridDhtAtomicUpdateRequest unmarshalling failed test
         failAtomicPut();
 
+        //Check that cache is not empty.
         readCnt.set(100);
 
         assert jcache(0).get(new TestKey("1")) != null;