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

[01/50] incubator-ignite git commit: IGNITE-1026 - Fixing dynamic cache start methods.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1085 7510aff0e -> 98d6c2681


IGNITE-1026 - Fixing dynamic cache start methods.


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

Branch: refs/heads/ignite-1085
Commit: bb73b66c6d29de578da3aed2654c2a887c97337c
Parents: eef2b37
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 18 15:49:06 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 18 15:49:06 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 16 +++++-----
 .../processors/cache/GridCacheProcessor.java    | 31 ++++++++++++--------
 .../datastructures/DataStructuresProcessor.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +--
 4 files changed, 30 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index e19d3d3..3ee260d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2265,7 +2265,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, true).get();
+            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, true, true).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2302,7 +2302,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         try {
             if (ctx.cache().cache(cacheCfg.getName()) == null)
-                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, false).get();
+                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, false, true).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2325,7 +2325,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, true).get();
+            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, true, true).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2349,10 +2349,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             IgniteInternalCache<Object, Object> cache = ctx.cache().cache(cacheCfg.getName());
 
             if (cache == null)
-                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false, true).get();
             else {
                 if (cache.configuration().getNearConfiguration() == null)
-                    ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+                    ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false, true).get();
             }
 
             return ctx.cache().publicJCache(cacheCfg.getName());
@@ -2372,7 +2372,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(null, cacheName, nearCfg, true).get();
+            ctx.cache().dynamicStartCache(null, cacheName, nearCfg, true, true).get();
 
             IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);
 
@@ -2399,10 +2399,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             IgniteInternalCache<Object, Object> internalCache = ctx.cache().cache(cacheName);
 
             if (internalCache == null)
-                ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+                ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false, true).get();
             else {
                 if (internalCache.configuration().getNearConfiguration() == null)
-                    ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+                    ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false, true).get();
             }
 
             IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/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 e6a0994..ac2d7b1 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
@@ -1458,6 +1458,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (clientNodeStart && !affNodeStart) {
                 if (nearCfg != null)
                     ccfg.setNearConfiguration(nearCfg);
+                else
+                    ccfg.setNearConfiguration(null);
             }
 
             CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
@@ -1746,7 +1748,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> createFromTemplate(String cacheName) {
         CacheConfiguration cfg = createConfigFromTemplate(cacheName);
 
-        return dynamicStartCache(cfg, cacheName, null, true);
+        return dynamicStartCache(cfg, cacheName, null, true, true);
     }
 
     /**
@@ -1762,7 +1764,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheConfiguration cfg = createConfigFromTemplate(cacheName);
 
-            return dynamicStartCache(cfg, cacheName, null, false);
+            return dynamicStartCache(cfg, cacheName, null, false, true);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(e);
@@ -1855,9 +1857,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         @Nullable CacheConfiguration ccfg,
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
-        boolean failIfExists
+        boolean failIfExists,
+        boolean failIfNotStarted
     ) {
-        return dynamicStartCache(ccfg, cacheName, nearCfg, CacheType.USER, failIfExists);
+        return dynamicStartCache(ccfg, cacheName, nearCfg, CacheType.USER, failIfExists, failIfNotStarted);
     }
 
     /**
@@ -1875,12 +1878,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
         CacheType cacheType,
-        boolean failIfExists
+        boolean failIfExists,
+        boolean failIfNotStarted
     ) {
         checkEmptyTransactions();
 
-        assert ccfg != null || nearCfg != null;
-
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
 
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
@@ -1947,9 +1949,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (desc != null && !desc.cancelled())
                 ccfg = desc.cacheConfiguration();
 
-            if (ccfg == null)
-                return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
-                    "(a cache with the given name is not started): " + cacheName));
+            if (ccfg == null) {
+                if (failIfNotStarted)
+                    return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
+                        "(a cache with the given name is not started): " + cacheName));
+                else
+                    return new GridFinishedFuture<>();
+            }
 
             if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
                 if (ccfg.getNearConfiguration() != null)
@@ -1961,6 +1967,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             req.deploymentId(desc.deploymentId());
             req.startCacheConfiguration(ccfg);
+
         }
 
         if (nearCfg != null)
@@ -2528,7 +2535,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         if (cache == null) {
-            dynamicStartCache(null, name, null, false);
+            dynamicStartCache(null, name, null, false, false);
 
             cache = jCacheProxies.get(masked);
         }
@@ -2644,7 +2651,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to get cache because it is a system cache: " + cacheName);
 
         if (cache == null) {
-            dynamicStartCache(null, cacheName, null, false).get();
+            dynamicStartCache(null, cacheName, null, false, failIfNotStarted).get();
 
             cache = jCacheProxies.get(masked);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 473a2ac..dcd22cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -819,7 +819,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         CacheConfiguration newCfg = cacheConfiguration(cfg, cacheName);
 
-        ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false).get();
+        ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false, true).get();
 
         return cacheName;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index 7905565..7d00417 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -143,7 +143,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
                 ccfg.setName(DYNAMIC_CACHE_NAME);
 
-                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true));
+                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true, true));
 
                 return null;
             }
@@ -203,7 +203,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
                 IgniteEx kernal = grid(ThreadLocalRandom.current().nextInt(nodeCount()));
 
-                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true));
+                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true, true));
 
                 return null;
             }


[20/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-591

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


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

Branch: refs/heads/ignite-1085
Commit: b84fdaa508592ad9950fd17718301ae4be22953f
Parents: bb8dcf9 f72b291
Author: sevdokimov <se...@jetbrains.com>
Authored: Sat Jul 4 16:07:19 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Sat Jul 4 16:07:19 2015 +0300

----------------------------------------------------------------------
 assembly/dependencies-fabric.xml                |   1 +
 bin/ignite.bat                                  |   8 +-
 bin/ignite.sh                                   |   6 +-
 bin/include/parseargs.bat                       |   1 +
 bin/include/parseargs.sh                        |   3 +
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../ignite/compute/ComputeTaskSplitAdapter.java |   2 +-
 .../configuration/CacheConfiguration.java       | 105 +++---
 .../configuration/NearCacheConfiguration.java   |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   1 -
 .../managers/communication/GridIoManager.java   | 173 ++++++---
 .../managers/communication/GridIoMessage.java   |  15 +-
 .../managers/communication/GridIoPolicy.java    |  32 +-
 .../discovery/GridDiscoveryManager.java         |  32 +-
 .../eventstorage/GridEventStorageManager.java   |   2 +-
 .../processors/cache/CacheOperationContext.java |  44 ++-
 .../internal/processors/cache/CacheType.java    |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  91 +++--
 .../processors/cache/GridCacheAtomicFuture.java |  12 +-
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../processors/cache/GridCacheIoManager.java    |  12 +-
 .../processors/cache/GridCacheMvccManager.java  |   8 +-
 .../processors/cache/GridCacheProcessor.java    |  62 ++-
 .../processors/cache/GridCacheProxyImpl.java    |  10 +-
 .../processors/cache/GridCacheSwapManager.java  | 257 ++++++++-----
 .../processors/cache/GridCacheUtils.java        |  42 +++
 .../processors/cache/IgniteCacheFutureImpl.java |  42 +++
 .../processors/cache/IgniteCacheProxy.java      |  38 +-
 .../GridDistributedTxFinishRequest.java         |  11 +-
 .../GridDistributedTxPrepareRequest.java        |   9 +-
 .../GridDistributedTxRemoteAdapter.java         |   3 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |   3 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   3 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   3 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   5 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  15 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 177 ++++++++-
 .../near/GridNearTxFinishRequest.java           |   3 +-
 .../cache/distributed/near/GridNearTxLocal.java |   3 +-
 .../distributed/near/GridNearTxRemote.java      |   5 +-
 .../cache/transactions/IgniteInternalTx.java    |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |  11 +-
 .../transactions/IgniteTxLocalAdapter.java      |   3 +-
 .../processors/clock/GridClockServer.java       |  21 +-
 .../datastructures/GridCacheAtomicLongImpl.java |  25 +-
 .../GridCacheAtomicSequenceImpl.java            |  11 +-
 .../GridCacheAtomicStampedImpl.java             |  21 +-
 .../GridCacheCountDownLatchImpl.java            |  16 +-
 .../internal/processors/igfs/IgfsContext.java   |   5 +-
 .../plugin/IgnitePluginProcessor.java           |   3 +-
 .../internal/util/future/IgniteFutureImpl.java  |  18 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../util/nio/GridNioMessageTracker.java         |  23 +-
 .../plugin/extensions/communication/IoPool.java |  42 +++
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   5 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   7 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  76 +++-
 .../startup/cmdline/CommandLineStartup.java     |   3 +-
 .../startup/cmdline/CommandLineTransformer.java |   9 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/spark/spark-config.xml |  46 +++
 modules/core/src/test/config/tests.properties   |   6 +-
 .../communication/GridIoManagerSelfTest.java    |   2 +-
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 ++++++++++++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ------
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 ++++++++++
 .../cache/IgniteInternalCacheTypesTest.java     |   3 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    | 147 ++++++++
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |  34 ++
 ...gniteCachePutRetryTransactionalSelfTest.java |  74 ++++
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   5 +-
 .../GridCachePartitionedFailoverSelfTest.java   |   5 +
 .../GridCacheEvictionFilterSelfTest.java        |   2 -
 .../GridP2PContinuousDeploymentSelfTest.java    |   2 -
 .../inmemory/GridTestSwapSpaceSpi.java          |   3 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 modules/core/src/test/resources/helloworld.gar  | Bin 6092 -> 0 bytes
 modules/core/src/test/resources/helloworld1.gar | Bin 6092 -> 0 bytes
 modules/core/src/test/resources/readme.txt      |   6 -
 modules/docker/Dockerfile                       |  55 +++
 modules/docker/README.txt                       |  11 +
 modules/docker/build_users_libs.sh              |  39 ++
 modules/docker/download_ignite.sh               |  49 +++
 modules/docker/execute.sh                       |  62 +++
 modules/docker/run.sh                           |  34 ++
 modules/extdata/p2p/pom.xml                     |   4 +-
 modules/extdata/uri/META-INF/ignite.xml         |  38 ++
 .../extdata/uri/modules/uri-dependency/pom.xml  |  42 +++
 .../deployment/uri/tasks/GarHelloWorldBean.java |  60 +++
 .../src/main/resources/gar-example.properties   |  18 +
 modules/extdata/uri/pom.xml                     |  62 ++-
 .../deployment/uri/tasks/GarHelloWorldTask.java |  81 ++++
 .../deployment/uri/tasks/gar-spring-bean.xml    |  29 ++
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   8 +-
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   4 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 -
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   5 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/kafka/pom.xml                           | 116 ++++++
 .../ignite/stream/kafka/KafkaStreamer.java      | 220 +++++++++++
 .../kafka/IgniteKafkaStreamerSelfTestSuite.java |  37 ++
 .../stream/kafka/KafkaEmbeddedBroker.java       | 378 +++++++++++++++++++
 .../kafka/KafkaIgniteStreamerSelfTest.java      | 227 +++++++++++
 .../ignite/stream/kafka/SimplePartitioner.java  |  53 +++
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 .../ignite/schema/model/PojoDescriptor.java     |   2 +
 .../apache/ignite/schema/model/PojoField.java   |   1 +
 .../parser/dialect/OracleMetadataDialect.java   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |  50 ++-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 +
 modules/spring/pom.xml                          |   2 +-
 .../util/spring/IgniteSpringHelperImpl.java     |  72 +++-
 .../IgniteExcludeInConfigurationTest.java       |  78 ++++
 .../org/apache/ignite/spring/sprint-exclude.xml |  57 +++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |  13 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    |   3 +-
 .../visor/commands/open/VisorOpenCommand.scala  | 319 ++++++++++++++++
 .../scala/org/apache/ignite/visor/visor.scala   | 230 +----------
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |   2 +
 .../commands/kill/VisorKillCommandSpec.scala    |   1 +
 .../commands/start/VisorStartCommandSpec.scala  |   1 +
 .../commands/tasks/VisorTasksCommandSpec.scala  |   1 +
 .../commands/vvm/VisorVvmCommandSpec.scala      |   1 +
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  15 +-
 166 files changed, 4204 insertions(+), 929 deletions(-)
----------------------------------------------------------------------



[06/50] incubator-ignite git commit: IGNITE-1026 - Dynamic cache start fix

Posted by sb...@apache.org.
IGNITE-1026 - Dynamic cache start 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/08ea4ccf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/08ea4ccf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/08ea4ccf

Branch: refs/heads/ignite-1085
Commit: 08ea4ccf251afc9d485c5409c4b43539d01c62fb
Parents: 7cf9d76
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 19 15:30:38 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 19 15:30:38 2015 -0700

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheProcessor.java       | 9 ---------
 .../store/CacheStoreSessionListenerAbstractSelfTest.java    | 1 -
 2 files changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/08ea4ccf/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 4f1a3e6..0f9247f 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
@@ -1957,17 +1957,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     return new GridFinishedFuture<>();
             }
 
-            if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
-                if (ccfg.getNearConfiguration() != null)
-                    return new GridFinishedFuture<>();
-                else
-                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start client cache " +
-                        "(local node is an affinity node for cache): " + cacheName));
-            }
-
             req.deploymentId(desc.deploymentId());
             req.startCacheConfiguration(ccfg);
-
         }
 
         if (nearCfg != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/08ea4ccf/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
index adac0b2..0634197 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
@@ -145,7 +145,6 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
         assertEquals(1, writeCnt.get());
         assertEquals(1, deleteCnt.get());
         assertEquals(0, reuseCnt.get());
-
     }
 
     /**


[22/50] incubator-ignite git commit: # IGNITE-591 Comment broken test

Posted by sb...@apache.org.
# IGNITE-591 Comment broken 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/e54f5729
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/e54f5729
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/e54f5729

Branch: refs/heads/ignite-1085
Commit: e54f5729b710c04d4a72b2ffabcc5c08cf21aaa2
Parents: 006209c
Author: sevdokimov <se...@jetbrains.com>
Authored: Sat Jul 4 17:34:58 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Sat Jul 4 17:34:58 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheJtaSelfTest.java  | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e54f5729/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index 29836ef..7302f7e 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -193,7 +193,12 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
         assertEquals(1, (int)cache2.get("key"));
     }
     
+    /**
+     *
+     */
     public void testUncompatibleTmLookup() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1094");
+        
         final IgniteEx ignite = grid(0);
         
         final CacheConfiguration cacheCfg = new CacheConfiguration();


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

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


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

Branch: refs/heads/ignite-1085
Commit: 9a51cdf8d051dd0a6c0159fe1a2bc4bf56201da7
Parents: bc29797 0f1b31a
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Jul 7 19:09:30 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Jul 7 19:09:30 2015 -0700

----------------------------------------------------------------------
 dev-tools/slurp.sh                              |   2 +-
 .../internal/interop/InteropIgnition.java       |  17 +-
 .../internal/interop/InteropProcessor.java      |   7 +
 .../cache/query/GridCacheQueryAdapter.java      |  35 ++++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 ++++++++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 ++++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ----
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 ++++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 ++++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ----
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 scripts/git-patch-prop.sh                       |   2 +-
 15 files changed, 386 insertions(+), 75 deletions(-)
----------------------------------------------------------------------



[11/50] incubator-ignite git commit: IGNITE-591 Make GridCacheXAResource shared.

Posted by sb...@apache.org.
IGNITE-591 Make GridCacheXAResource shared.


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

Branch: refs/heads/ignite-1085
Commit: 4b06e77c8e770f9b9ef8b34b59a69ec47da615c7
Parents: 1a21477
Author: sevdokimov <se...@jetbrains.com>
Authored: Wed Jul 1 10:08:13 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Wed Jul 1 10:08:13 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/jta/CacheJtaManager.java       |  4 ++--
 .../processors/cache/jta/GridCacheXAResource.java   | 16 ++++++----------
 .../processors/cache/GridCacheJtaSelfTest.java      |  2 --
 3 files changed, 8 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b06e77c/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index 9af29d6..2a078dc 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -30,7 +30,7 @@ import javax.transaction.*;
  */
 public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** */
-    private final ThreadLocal<GridCacheXAResource> xaRsrc = new ThreadLocal<>();
+    private final static ThreadLocal<GridCacheXAResource> xaRsrc = new ThreadLocal<>();
 
     /** */
     private TransactionManager jtaTm;
@@ -89,7 +89,7 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
                             );
                         }
 
-                        rsrc = new GridCacheXAResource(tx, cctx);
+                        rsrc = new GridCacheXAResource(tx, cctx.kernalContext());
 
                         if (!jtaTx.enlistResource(rsrc))
                             throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b06e77c/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
index 442cdde..ad233e5 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.cache.jta;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.transactions.*;
@@ -41,9 +41,6 @@ public final class GridCacheXAResource implements XAResource {
     /** */
     private static final Xid[] NO_XID = new Xid[] {};
 
-    /** Context. */
-    private GridCacheContext cctx;
-
     /** Cache transaction. */
     private IgniteInternalTx cacheTx;
 
@@ -52,17 +49,16 @@ public final class GridCacheXAResource implements XAResource {
 
     /**
      * @param cacheTx Cache jta.
-     * @param cctx Cache context.
+     * @param ctx Kernal context.
      */
-    public GridCacheXAResource(IgniteInternalTx cacheTx, GridCacheContext cctx) {
+    public GridCacheXAResource(IgniteInternalTx cacheTx, GridKernalContext ctx) {
         assert cacheTx != null;
-        assert cctx != null;
+        assert ctx != null;
 
-        this.cctx = cctx;
         this.cacheTx = cacheTx;
 
         if (log == null)
-            log = U.logger(cctx.kernalContext(), logRef, GridCacheXAResource.class);
+            log = U.logger(ctx, logRef, GridCacheXAResource.class);
     }
 
     /** {@inheritDoc} */
@@ -229,7 +225,7 @@ public final class GridCacheXAResource implements XAResource {
 
         GridCacheXAResource other = (GridCacheXAResource)xar;
 
-        return cctx == other.cctx;
+        return cacheTx == other.cacheTx;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4b06e77c/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index b299f8d..37e9ed2 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -151,8 +151,6 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
      */
     @SuppressWarnings("ConstantConditions")
     public void testJtaTwoCaches() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-591");
-
         UserTransaction jtaTx = jotm.getUserTransaction();
 
         IgniteEx ignite = grid(0);


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

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


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

Branch: refs/heads/ignite-1085
Commit: 2f089f1bda24b0e54735e5f8dccfc372647e1c54
Parents: d6dd707 bb73b66
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 16:03:40 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 16:03:40 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 16 +++++-----
 .../processors/cache/GridCacheProcessor.java    | 31 ++++++++++++--------
 .../datastructures/DataStructuresProcessor.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +--
 4 files changed, 30 insertions(+), 23 deletions(-)
----------------------------------------------------------------------



[44/50] incubator-ignite git commit: release notes

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


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

Branch: refs/heads/ignite-1085
Commit: f13f5946beaae7115f2ca5796988624ac36c193a
Parents: 064d079
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:52:13 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:52:13 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f13f5946/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index bcfed27..ec8c4e6 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -6,6 +6,7 @@ Apache Ignite In-Memory Data Fabric 1.3
 
 * Added auto-retries for cache operations in recoverable cases.
 * Fixed several issues with JTA integration.
+* Fixed issue with GAR files in source release.
 * Stability fixes for TCP discovery SPI.
 * Stability fixes for onheap and offheap SQL queries.
 * Bug fixes in In-Memory Accelerator For Apache Hadoop.


[33/50] incubator-ignite git commit: Fixed notes

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


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

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

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


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

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

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

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

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

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

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

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

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

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


[40/50] incubator-ignite git commit: Redundant dependencies removed from ignite-kafka module

Posted by sb...@apache.org.
Redundant dependencies removed from ignite-kafka module


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

Branch: refs/heads/ignite-1085
Commit: 4c9d8c2e4d5421d4088a3f3bfe3f17df589a4f0f
Parents: 23dc8fc
Author: agura <ag...@gridgain.com>
Authored: Wed Jul 8 17:09:19 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jul 8 17:09:56 2015 +0300

----------------------------------------------------------------------
 modules/kafka/pom.xml | 11 -----------
 1 file changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4c9d8c2e/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index d0e9cd9..c492100 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -87,17 +87,6 @@
         </dependency>
 
         <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>ignite-shmem</artifactId>
-            <version>1.0.0</version>
-        </dependency>
-
-        <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-spring</artifactId>
             <version>${project.version}</version>


[41/50] incubator-ignite git commit: moved link to the project site to the top of the log

Posted by sb...@apache.org.
moved link to the project site to the top of the log


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

Branch: refs/heads/ignite-1085
Commit: 459d7022f8685858eb01792f2c99db5f5efc8e9a
Parents: 82f4992
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:39:09 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:39:09 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/459d7022/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 30931fa..c12d2cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1486,10 +1486,12 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     "  /  _/ ___/ |/ /  _/_  __/ __/ ",
                     " _/ // (7 7    // /  / / / _/   ",
                     "/___/\\___/_/|_/___/ /_/ /___/  ",
-                    " ",
+                    "",
                     ver,
                     COPYRIGHT,
                     "",
+                    "Ignite documentation: " + "http://" + SITE,
+                    "",
                     "Quiet mode.");
 
                 if (fileName != null)
@@ -1508,7 +1510,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                         ">>> /___/\\___/_/|_/___/ /_/ /___/   " + NL +
                         ">>> " + NL +
                         ">>> " + ver + NL +
-                        ">>> " + COPYRIGHT + NL
+                        ">>> " + COPYRIGHT + NL +
+                        ">>> " + NL +
+                        ">>> Ignite documentation: " + "http://" + SITE + NL
                 );
             }
         }
@@ -1557,8 +1561,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     ">>> Local node addresses: " + U.addressesAsString(locNode) + NL +
                     ">>> Local ports: " + sb + NL;
 
-            str += ">>> Ignite documentation: http://" + SITE + NL;
-
             log.info(str);
         }
     }


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

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


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

Branch: refs/heads/ignite-1085
Commit: fe92bfec6ee1eaf55df5d1654f51b9b143518a09
Parents: 9adc180 b29ff1c
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 24 15:49:23 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 24 15:49:23 2015 -0700

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../configuration/IgniteReflectionFactory.java  |  81 +++-
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../processors/cache/GridCacheIoManager.java    |  64 ++-
 .../GridCachePartitionExchangeManager.java      |  70 ++-
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   9 +-
 .../GridDhtPartitionsExchangeFuture.java        |  95 +++-
 .../datastructures/DataStructuresProcessor.java |  64 +--
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../processors/task/GridTaskProcessor.java      |  23 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  61 ++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  68 ++-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../GridTaskFailoverAffinityRunTest.java        | 170 +++++++
 .../CacheReadThroughAtomicRestartSelfTest.java  |  32 ++
 ...heReadThroughLocalAtomicRestartSelfTest.java |  32 ++
 .../CacheReadThroughLocalRestartSelfTest.java   |  32 ++
 ...dThroughReplicatedAtomicRestartSelfTest.java |  32 ++
 ...cheReadThroughReplicatedRestartSelfTest.java |  32 ++
 .../cache/CacheReadThroughRestartSelfTest.java  | 133 ++++++
 .../GridCacheAbstractFailoverSelfTest.java      |   6 +-
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++++++++++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  37 ++
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   4 +-
 .../GridCacheSetAbstractSelfTest.java           |  22 +-
 .../IgniteDataStructureWithJobTest.java         | 111 +++++
 ...ridCachePartitionNotLoadedEventSelfTest.java |  82 ++++
 .../distributed/IgniteCache150ClientsTest.java  | 189 ++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../distributed/IgniteCacheManyClientsTest.java |   2 +
 .../IgniteCacheTxMessageRecoveryTest.java       |   5 +
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   5 -
 ...achePartitionedMultiNodeFullApiSelfTest.java |  49 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 +
 .../IgniteCacheTxStoreSessionTest.java          |   4 +
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   1 -
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   1 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   8 +
 .../testsuites/IgniteClientTestSuite.java       |  38 ++
 .../testsuites/IgniteComputeGridTestSuite.java  |   1 +
 .../ignite/util/TestTcpCommunicationSpi.java    |  21 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |  80 +---
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 ++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +-
 .../internal/processors/hadoop/HadoopUtils.java | 237 ----------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   | 241 ++++++++++
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 +++-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 ++++++++
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 +
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 .../IgniteWebSessionSelfTestSuite.java          |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 117 files changed, 2759 insertions(+), 683 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe92bfec/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe92bfec/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------


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

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


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

Branch: refs/heads/ignite-1085
Commit: 25c349f47f70c9824c8d4ba9c4ee679ff597051e
Parents: 5dc2d2b 9d0aa6f
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jul 7 16:01:47 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jul 7 16:01:47 2015 +0300

----------------------------------------------------------------------
 .../cache/eviction/fifo/FifoEvictionPolicy.java  |  5 -----
 .../cache/eviction/lru/LruEvictionPolicy.java    |  5 -----
 .../eviction/sorted/SortedEvictionPolicy.java    | 19 +++++--------------
 ...cheConcurrentEvictionConsistencySelfTest.java | 15 +++++++++++++--
 4 files changed, 18 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



[29/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/839fe79f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/839fe79f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/839fe79f

Branch: refs/heads/ignite-1085
Commit: 839fe79fa5ff0ee91b528b43d6a7834aa1ea719f
Parents: cd01e4c
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 06:16:48 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 06:16:48 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/jta/CacheJtaManager.java   | 24 ++++++++++----------
 1 file changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/839fe79f/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index 09d856b..ace5fe7 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -35,7 +35,7 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
     private final ThreadLocal<GridCacheXAResource> xaRsrc = new ThreadLocal<>();
 
     /** */
-    private volatile TransactionManager jtaTm;
+    private TransactionManager jtaTm;
 
     /** */
     private final AtomicReference<CacheTmLookup> tmLookupRef = new AtomicReference<>();
@@ -46,7 +46,7 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
 
         if (cctx.txConfig() != null) {
             String txLookupClsName = cctx.txConfig().getTxManagerLookupClassName();
-            
+
             if (txLookupClsName != null)
                 tmLookupRef.set(createTmLookup(txLookupClsName));
         }
@@ -55,7 +55,7 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
         CacheTmLookup tmLookup = tmLookupRef.get();
-        
+
         if (tmLookup instanceof LifecycleAware)
             ((LifecycleAware)tmLookup).stop();
     }
@@ -70,10 +70,10 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
             CacheTmLookup res = (CacheTmLookup)cls.newInstance();
 
             cctx.kernalContext().resource().injectGeneric(res);
-            
+
             if (res instanceof LifecycleAware)
                 ((LifecycleAware)res).start();
-            
+
             return res;
         }
         catch (Exception e) {
@@ -86,10 +86,10 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
         if (jtaTm == null) {
             try {
                 CacheTmLookup tmLookup = tmLookupRef.get();
-                
+
                 if (tmLookup == null)
                     return;
-                
+
                 jtaTm = tmLookup.getTm();
             }
             catch (Exception e) {
@@ -144,23 +144,23 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** {@inheritDoc} */
     @Override public void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
         String cacheLookupClsName = cfg.getTransactionManagerLookupClassName();
-        
+
         if (cacheLookupClsName != null) {
             CacheTmLookup tmLookup = tmLookupRef.get();
-            
+
             if (tmLookup == null) {
                 tmLookup = createTmLookup(cacheLookupClsName);
-                
+
                 if (tmLookupRef.compareAndSet(null, tmLookup))
                     return;
-                
+
                 tmLookup = tmLookupRef.get();
             }
 
             if (!cacheLookupClsName.equals(tmLookup.getClass().getName()))
                 throw new IgniteCheckedException("Failed to start cache with CacheTmLookup that specified in cache " +
                     "configuration, because node uses another CacheTmLookup [cache" + cfg.getName() +
-                    ", tmLookupClassName=" + cacheLookupClsName + ", tmLookupUsedByNode=" 
+                    ", tmLookupClassName=" + cacheLookupClsName + ", tmLookupUsedByNode="
                     + tmLookup.getClass().getName() + ']');
         }
     }


[34/50] incubator-ignite git commit: Apache license added to ignite-kafka modeule

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


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

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

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


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


[16/50] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by sb...@apache.org.
IGNITE-1026 - Count down latch 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/6f50ad9f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/6f50ad9f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/6f50ad9f

Branch: refs/heads/ignite-1085
Commit: 6f50ad9f2d7a0e049863f235104c65c86ee8c5ad
Parents: 8ff3619
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jul 2 16:05:03 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jul 2 16:05:03 2015 -0700

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryHandler.java |  4 +--
 .../datastructures/DataStructuresProcessor.java | 31 +++++++++++++++++---
 .../GridCacheCountDownLatchImpl.java            | 12 ++++----
 3 files changed, 36 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f50ad9f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index ff2905f..e059760 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -353,8 +353,8 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
         Iterable<CacheEntryEvent<? extends K, ? extends V>> evts = F.viewReadOnly(entries,
             new C1<CacheContinuousQueryEntry, CacheEntryEvent<? extends K, ? extends V>>() {
                 @Override public CacheEntryEvent<? extends K, ? extends V> apply(CacheContinuousQueryEntry e) {
-                    return new CacheContinuousQueryEvent<K, V>(cache, cctx, e);
-                };
+                    return new CacheContinuousQueryEvent<>(cache, cctx, e);
+                }
             }
         );
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f50ad9f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 4d2ecbe..5c171e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1056,7 +1056,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         dsView.remove(key);
 
                         tx.commit();
-                    } else
+                    }
+                    else
                         tx.setRollbackOnly();
 
                     return null;
@@ -1147,19 +1148,41 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         GridCacheInternalKey key = evt.getKey();
 
                         // Notify latch on changes.
-                        GridCacheRemovable latch = dsMap.get(key);
+                        final GridCacheRemovable latch = dsMap.get(key);
 
                         GridCacheCountDownLatchValue val = (GridCacheCountDownLatchValue)val0;
 
                         if (latch instanceof GridCacheCountDownLatchEx) {
-                            GridCacheCountDownLatchEx latch0 = (GridCacheCountDownLatchEx)latch;
+                            final GridCacheCountDownLatchEx latch0 = (GridCacheCountDownLatchEx)latch;
 
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
                                 dsMap.remove(key);
 
-                                latch.onRemoved();
+                                IgniteInternalFuture<?> removeFut = ctx.closure().runLocalSafe(new GPR() {
+                                    @Override public void run() {
+                                        try {
+                                            removeCountDownLatch(latch0.name());
+                                        }
+                                        catch (IgniteCheckedException e) {
+                                            U.error(log, "Failed to remove count down latch: " + latch0.name(), e);
+                                        }
+                                    }
+                                });
+
+                                removeFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                                    @Override public void apply(IgniteInternalFuture<?> f) {
+                                        try {
+                                            f.get();
+                                        }
+                                        catch (IgniteCheckedException e) {
+                                            U.error(log, "Failed to remove count down latch: " + latch0.name(), e);
+                                        }
+
+                                        latch.onRemoved();
+                                    }
+                                });
                             }
                         }
                         else if (latch != null) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f50ad9f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index a5353d8..2df6015 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -288,11 +288,13 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
     /** {@inheritDoc} */
     @Override public void close() {
-        try {
-            ctx.kernalContext().dataStructures().removeCountDownLatch(name);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        if (!rmvd) {
+            try {
+                ctx.kernalContext().dataStructures().removeCountDownLatch(name);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
         }
     }
 


[50/50] incubator-ignite git commit: ignite-1085: merging

Posted by sb...@apache.org.
ignite-1085: merging


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

Branch: refs/heads/ignite-1085
Commit: 98d6c26811c5f9efe4b8ce4d40e9b9fab5d82fff
Parents: 7510aff bee6f68
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 9 17:52:13 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Jul 9 17:52:13 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  12 ++
 .../src/main/java/org/apache/ignite/Ignite.java |   2 +-
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../apache/ignite/internal/IgniteKernal.java    |  32 +--
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheIoManager.java    |   8 +-
 .../processors/cache/GridCacheProcessor.java    | 118 ++++-------
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../distributed/near/GridNearGetFuture.java     |   4 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../continuous/CacheContinuousQueryHandler.java |   4 +-
 .../datastructures/DataStructuresProcessor.java |  39 +++-
 .../GridCacheCountDownLatchImpl.java            |  15 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 -
 ...cheStoreSessionListenerAbstractSelfTest.java |   1 -
 .../cache/CacheFutureExceptionSelfTest.java     | 161 +++++++--------
 .../IgniteCacheConfigurationTemplateTest.java   |  26 +--
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  16 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |   5 +-
 .../IgniteClientDataStructuresAbstractTest.java | 109 +++++++---
 .../IgniteCountDownLatchAbstractSelfTest.java   |  12 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   4 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 -
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   |  72 ++++++-
 .../cache/jta/GridCacheXAResource.java          |  16 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |  52 +++--
 .../GridTmLookupLifecycleAwareSelfTest.java     |  29 ++-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/kafka/pom.xml                           |  11 -
 .../util/spring/IgniteSpringHelperImpl.java     |  10 +-
 .../IgniteExcludeInConfigurationTest.java       |   5 +-
 .../org/apache/ignite/spring/sprint-exclude.xml |  19 ++
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 .../visor/commands/open/VisorOpenCommand.scala  |   2 +-
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 modules/yardstick/config/ignite-base-config.xml |  23 +++
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  55 +++++
 .../ignite/yardstick/cache/model/Person2.java   |  67 ++++++
 .../ignite/yardstick/cache/model/Person8.java   | 109 ++++++++++
 49 files changed, 1168 insertions(+), 376 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/98d6c268/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 0f883c1,de1eac2..4072655
--- 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
@@@ -1916,9 -1918,7 +1915,7 @@@ public class GridCacheProcessor extend
      ) {
          checkEmptyTransactions();
  
-         assert ccfg != null || nearCfg != null;
- 
 -        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
 +        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(cacheName));
  
          DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
  
@@@ -2561,12 -2556,17 +2554,17 @@@
          if (log.isDebugEnabled())
              log.debug("Getting cache for name: " + name);
  
-         IgniteCache<K, V> jcache = (IgniteCache<K, V>)jCacheProxies.get(U.maskNull(name));
 -        String masked = maskNull(name);
++        String masked = U.maskNull(name);
  
-         if (jcache == null)
-             jcache = startJCache(name, true);
+         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
  
-         return jcache == null ? null : ((IgniteCacheProxy<K, V>)jcache).internalProxy();
+         if (cache == null) {
+             dynamicStartCache(null, name, null, false, true).get();
+ 
+             cache = jCacheProxies.get(masked);
+         }
+ 
+         return cache == null ? null : (IgniteInternalCache<K, V>)cache.internalProxy();
      }
  
      /**


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

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


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

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

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



[13/50] incubator-ignite git commit: IGNITE-1026 - Dynamic cache start fixes

Posted by sb...@apache.org.
IGNITE-1026 - Dynamic cache start fixes


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

Branch: refs/heads/ignite-1085
Commit: 8ff3619bae65018958c61a3299a415e595dd16c4
Parents: 378e344
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 1 18:51:51 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 1 18:51:51 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  2 +-
 .../IgniteCacheConfigurationTemplateTest.java   | 26 +++-----------------
 .../cache/IgniteDynamicCacheStartSelfTest.java  | 12 ++-------
 .../IgniteDynamicClientCacheStartSelfTest.java  |  5 ++--
 4 files changed, 9 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 209946b..7103b1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -308,7 +308,7 @@ public interface Ignite extends AutoCloseable {
      * @param name Cache name.
      * @return Instance of the cache for the specified name.
      */
-    public <K, V> IgniteCache<K, V> cache(@Nullable String name);
+    @Nullable public <K, V> IgniteCache<K, V> cache(@Nullable String name);
 
     /**
      * Gets grid transactions facade.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
index 937a3b4..1085411 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
@@ -420,29 +420,9 @@ public class IgniteCacheConfigurationTemplateTest extends GridCommonAbstractTest
                 }
             }, IllegalStateException.class, null);
 
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    ignite.cache(TEMPLATE1);
-
-                    return null;
-                }
-            }, IllegalArgumentException.class, null);
-
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    ignite.cache(TEMPLATE2);
-
-                    return null;
-                }
-            }, IllegalArgumentException.class, null);
-
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    ignite.cache(TEMPLATE3);
-
-                    return null;
-                }
-            }, IllegalArgumentException.class, null);
+            assertNull(ignite.cache(TEMPLATE1));
+            assertNull(ignite.cache(TEMPLATE2));
+            assertNull(ignite.cache(TEMPLATE3));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index e7b0ba5..cd19703 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -310,11 +310,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                 f.get();
 
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    return kernal0.cache(DYNAMIC_CACHE_NAME);
-                }
-            }, IllegalArgumentException.class, null);
+            assertNull(kernal0.cache(DYNAMIC_CACHE_NAME));
 
             GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
@@ -368,11 +364,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                     f.get();
 
-                GridTestUtils.assertThrows(log, new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        return kernal0.cache(DYNAMIC_CACHE_NAME);
-                    }
-                }, IllegalArgumentException.class, null);
+                assertNull(kernal0.cache(DYNAMIC_CACHE_NAME));
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
index 24935c7..abada13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import javax.cache.*;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
@@ -139,7 +140,7 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
 
                 return null;
             }
-        }, IgniteException.class, null);
+        }, CacheException.class, null);
 
         checkCache(ignite1, cacheName, false, false);
 
@@ -149,7 +150,7 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
 
                 return null;
             }
-        }, IgniteException.class, null);
+        }, CacheException.class, null);
 
         checkCache(ignite1, cacheName, false, false);
     }


[47/50] incubator-ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10416

Posted by sb...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10416


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

Branch: refs/heads/ignite-1085
Commit: d04c1042ec120027c1ddc84d672726db7b7135b0
Parents: 8cc75fc c134dcf
Author: Andrey <an...@gridgain.com>
Authored: Thu Jul 9 17:04:20 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Thu Jul 9 17:04:20 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  12 ++
 .../src/main/java/org/apache/ignite/Ignite.java |   2 +-
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../apache/ignite/internal/IgniteKernal.java    |  32 +--
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheIoManager.java    |   8 +-
 .../processors/cache/GridCacheProcessor.java    | 118 ++++-------
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../distributed/near/GridNearGetFuture.java     |   4 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../continuous/CacheContinuousQueryHandler.java |   4 +-
 .../datastructures/DataStructuresProcessor.java |  39 +++-
 .../GridCacheCountDownLatchImpl.java            |  15 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  20 ++
 .../tcp/internal/TcpDiscoveryNode.java          |   2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   8 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |  10 +-
 ...cheStoreSessionListenerAbstractSelfTest.java |   1 -
 .../cache/CacheFutureExceptionSelfTest.java     | 161 +++++++--------
 .../IgniteCacheConfigurationTemplateTest.java   |  26 +--
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  16 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |   5 +-
 .../IgniteClientDataStructuresAbstractTest.java | 109 +++++++---
 .../IgniteCountDownLatchAbstractSelfTest.java   |  12 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java |   8 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   4 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  38 ++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   3 +
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 -
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   |  72 ++++++-
 .../cache/jta/GridCacheXAResource.java          |  16 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |  52 +++--
 .../GridTmLookupLifecycleAwareSelfTest.java     |  29 ++-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/kafka/pom.xml                           |  11 -
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 modules/yardstick/config/ignite-base-config.xml |  23 +++
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  55 +++++
 .../ignite/yardstick/cache/model/Person2.java   |  67 ++++++
 .../ignite/yardstick/cache/model/Person8.java   | 109 ++++++++++
 51 files changed, 1219 insertions(+), 376 deletions(-)
----------------------------------------------------------------------



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

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


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

Branch: refs/heads/ignite-1085
Commit: d38ad8ce2fe85ac61bd050fa696348f1e6cc29fa
Parents: 6f50ad9 ea90d86
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jul 2 16:05:12 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jul 2 16:05:12 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../managers/communication/GridIoManager.java   | 124 +++++++--
 .../managers/communication/GridIoMessage.java   |  15 +-
 .../managers/communication/GridIoPolicy.java    |  32 +--
 .../eventstorage/GridEventStorageManager.java   |   2 +-
 .../processors/cache/CacheOperationContext.java |  44 +++-
 .../internal/processors/cache/CacheType.java    |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  91 ++++---
 .../processors/cache/GridCacheAtomicFuture.java |  12 +-
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../processors/cache/GridCacheIoManager.java    |  12 +-
 .../processors/cache/GridCacheMvccManager.java  |   8 +-
 .../processors/cache/GridCacheProxyImpl.java    |  10 +-
 .../processors/cache/GridCacheSwapManager.java  | 257 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |  42 +++
 .../processors/cache/IgniteCacheProxy.java      |  36 ++-
 .../GridDistributedTxFinishRequest.java         |  11 +-
 .../GridDistributedTxPrepareRequest.java        |   9 +-
 .../GridDistributedTxRemoteAdapter.java         |   3 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |   3 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   3 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   3 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   5 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  15 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 177 +++++++++++--
 .../near/GridNearTxFinishRequest.java           |   3 +-
 .../cache/distributed/near/GridNearTxLocal.java |   3 +-
 .../distributed/near/GridNearTxRemote.java      |   5 +-
 .../cache/transactions/IgniteInternalTx.java    |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |  11 +-
 .../transactions/IgniteTxLocalAdapter.java      |   3 +-
 .../datastructures/GridCacheAtomicLongImpl.java |  25 +-
 .../GridCacheAtomicSequenceImpl.java            |  11 +-
 .../GridCacheAtomicStampedImpl.java             |  21 +-
 .../GridCacheCountDownLatchImpl.java            |  16 +-
 .../internal/processors/igfs/IgfsContext.java   |   5 +-
 .../plugin/IgnitePluginProcessor.java           |   3 +-
 .../plugin/extensions/communication/IoPool.java |  42 +++
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   5 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   5 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   2 +-
 .../communication/GridIoManagerSelfTest.java    |   2 +-
 .../cache/IgniteInternalCacheTypesTest.java     |   3 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    | 147 +++++++++++
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |  34 +++
 ...gniteCachePutRetryTransactionalSelfTest.java |  74 ++++++
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   5 +-
 .../GridCacheEvictionFilterSelfTest.java        |   2 -
 .../inmemory/GridTestSwapSpaceSpi.java          |   3 +-
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   8 +-
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   4 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 -
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   5 +
 modules/urideploy/pom.xml                       |  14 -
 60 files changed, 1053 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d38ad8ce/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------


[03/50] incubator-ignite git commit: IGNITE-1026 - Data structures fix

Posted by sb...@apache.org.
IGNITE-1026 - Data structures 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/1419d39b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1419d39b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1419d39b

Branch: refs/heads/ignite-1085
Commit: 1419d39ba80da9f85b28c708589f74dcec6514d5
Parents: 2f089f1
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 16:54:51 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 16:54:51 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java     | 2 +-
 .../processors/datastructures/DataStructuresProcessor.java       | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1419d39b/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 ac2d7b1..4f1a3e6 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
@@ -2535,7 +2535,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         if (cache == null) {
-            dynamicStartCache(null, name, null, false, false);
+            dynamicStartCache(null, name, null, false, true).get();
 
             cache = jCacheProxies.get(masked);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1419d39b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index dcd22cd..26e2f6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -177,7 +177,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         if (initLatch.getCount() > 0) {
             initFailed = true;
-            
+
             initLatch.countDown();
         }
 
@@ -890,7 +890,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
             String cacheName = ((CollectionInfo)oldInfo.info).cacheName;
 
-            GridCacheContext cacheCtx = ctx.cache().internalCache(cacheName).context();
+            GridCacheContext cacheCtx = ctx.cache().getOrStartCache(cacheName).context();
 
             return c.applyx(cacheCtx);
         }


[31/50] incubator-ignite git commit: Added benchmarks.

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


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

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

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


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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
new file mode 100644
index 0000000..0b2a0bf
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person1.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache.model;
+
+import org.apache.ignite.cache.query.annotations.*;
+
+import java.io.*;
+
+/**
+ * Value used for indexed put test.
+ */
+public class Person1 implements Serializable {
+    /** Value. */
+    @QuerySqlField(index = true)
+    private int val1;
+
+    /**
+     * Empty constructor.
+     */
+    public Person1() {
+        // No-op.
+    }
+
+    /**
+     * Constructs.
+     *
+     * @param val1 Indexed value.
+     */
+    public Person1(int val1) {
+        this.val1 = val1;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal1() {
+        return val1;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val1 Value.
+     */
+    public void setVal1(int val1) {
+        this.val1 = val1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        return this == o || (o instanceof Person1) && val1 == ((Person1)o).val1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return val1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "IndexedValue1 [val1=" + val1 + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
new file mode 100644
index 0000000..3b1cb9e
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person2.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache.model;
+
+import org.apache.ignite.cache.query.annotations.*;
+
+import java.io.*;
+
+/**
+ * Value used for indexed put test.
+ */
+public class Person2 implements Serializable {
+    /** Value 1. */
+    @QuerySqlField(index = true)
+    private int val1;
+
+    /** Value 2. */
+    @QuerySqlField(index = true)
+    private int val2;
+
+    /**
+     * Empty constructor.
+     */
+    public Person2() {
+        // No-op.
+    }
+
+    /**
+     * Constructs.
+     */
+    public Person2(int val1, int val2) {
+        this.val1 = val1;
+        this.val2 = val2;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal1() {
+        return val1;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val1 Value.
+     */
+    public void setVal1(int val1) {
+        this.val1 = val1;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal2() {
+        return val2;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val2 Value.
+     */
+    public void setVal2(int val2) {
+        this.val2 = val2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Person2 value2 = (Person2)o;
+
+        return val1 == value2.val1 && val2 == value2.val2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return 31 * val1 + val2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "IndexedValue2 [val1=" + val1 + ", val2=" + val2 + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/946b08ba/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
new file mode 100644
index 0000000..b8e3f7c
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/model/Person8.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache.model;
+
+import org.apache.ignite.cache.query.annotations.*;
+
+import java.io.*;
+
+/**
+ * Value used for indexed put test.
+ */
+public class Person8 implements Serializable {
+    /** Value 1. */
+    @QuerySqlField(index = true)
+    private int val1;
+
+    /** Value 2. */
+    @QuerySqlField(index = true)
+    private int val2;
+
+    /** Value 3. */
+    @QuerySqlField(index = true)
+    private int val3;
+
+    /** Value 4. */
+    @QuerySqlField(index = true)
+    private int val4;
+
+    /** Value 5. */
+    @QuerySqlField(index = true)
+    private int val5;
+
+    /** Value 6. */
+    @QuerySqlField(index = true)
+    private int val6;
+
+    /** Value 7. */
+    @QuerySqlField(index = true)
+    private int val7;
+
+    /** Value 8. */
+    @QuerySqlField(index = true)
+    private int val8;
+
+    /**
+     * Empty constructor.
+     */
+    public Person8() {
+        // No-op.
+    }
+
+    /**
+     * Constructs.
+     */
+    public Person8(int val1, int val2, int val3, int val4, int val5, int val6, int val7, int val8) {
+        this.val1 = val1;
+        this.val2 = val2;
+        this.val3 = val3;
+        this.val4 = val4;
+        this.val5 = val5;
+        this.val6 = val6;
+        this.val7 = val7;
+        this.val8 = val8;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal1() {
+        return val1;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val1 Value.
+     */
+    public void setVal1(int val1) {
+        this.val1 = val1;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal2() {
+        return val2;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val2 Value.
+     */
+    public void setVal2(int val2) {
+        this.val2 = val2;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal3() {
+        return val3;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val3 Value.
+     */
+    public void setVal3(int val3) {
+        this.val3 = val3;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal4() {
+        return val4;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val4 Value.
+     */
+    public void setVal4(int val4) {
+        this.val4 = val4;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal5() {
+        return val5;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val5 Value.
+     */
+    public void setVal5(int val5) {
+        this.val5 = val5;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal6() {
+        return val6;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val6 Value.
+     */
+    public void setVal6(int val6) {
+        this.val6 = val6;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal7() {
+        return val7;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val7 Value.
+     */
+    public void setVal7(int val7) {
+        this.val7 = val7;
+    }
+
+    /**
+     * @return Get value.
+     */
+    public int getVal8() {
+        return val8;
+    }
+
+    /**
+     * Set value.
+     *
+     * @param val8 Value.
+     */
+    public void setVal8(int val8) {
+        this.val8 = val8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Person8 p8 = (Person8)o;
+
+        return val1 == p8.val1 && val2 == p8.val2 && val3 == p8.val3 && val4 == p8.val4
+            && val5 == p8.val5 && val6 == p8.val6 && val7 == p8.val7 && val8 == p8.val8;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = val1;
+
+        result = 31 * result + val2;
+        result = 31 * result + val3;
+        result = 31 * result + val4;
+        result = 31 * result + val5;
+        result = 31 * result + val6;
+        result = 31 * result + val7;
+        result = 31 * result + val8;
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "Person8 [val1=" + val1 + ", val2=" + val2 + ", val3=" + val3 + ", val4=" + val4 + ", val5=" + val5 +
+            ", val6=" + val6 + ", val7=" + val7 + ", val8=" + val8 +']';
+    }
+}


[45/50] incubator-ignite git commit: release notes

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


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

Branch: refs/heads/ignite-1085
Commit: 0a569b8acfa4918dcd98a9fa0e4873f1e5b5737a
Parents: f13f594
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:55:58 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:55:58 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a569b8a/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index ec8c4e6..0e22f1f 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -6,6 +6,7 @@ Apache Ignite In-Memory Data Fabric 1.3
 
 * Added auto-retries for cache operations in recoverable cases.
 * Fixed several issues with JTA integration.
+* Fixed several issues with Hibernate L2 cache.
 * Fixed issue with GAR files in source release.
 * Stability fixes for TCP discovery SPI.
 * Stability fixes for onheap and offheap SQL queries.


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

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


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

Branch: refs/heads/ignite-1085
Commit: 378e344aba5048e36d86ca0944435586d45b7032
Parents: 7c935a5 d3783a1
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 1 15:26:40 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 1 15:26:40 2015 -0700

----------------------------------------------------------------------
 assembly/dependencies-fabric.xml                |   1 +
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 .../ClientAbstractConnectivitySelfTest.java     |   4 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../org/apache/ignite/cluster/ClusterGroup.java |  18 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   2 +
 .../configuration/CacheConfiguration.java       | 105 +--
 .../configuration/NearCacheConfiguration.java   |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |   5 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  50 +-
 .../cluster/IgniteClusterAsyncImpl.java         |  12 +-
 .../managers/communication/GridIoManager.java   |  49 +-
 .../discovery/GridDiscoveryManager.java         |  32 +-
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../processors/cache/GridCacheProcessor.java    |  62 +-
 .../processors/cache/IgniteCacheFutureImpl.java |  42 +
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../processors/clock/GridClockServer.java       |  21 +-
 .../processors/plugin/CachePluginManager.java   |  10 +-
 .../processors/rest/GridRestProcessor.java      |   4 +-
 .../handlers/task/GridTaskCommandHandler.java   |  12 +-
 .../processors/task/GridTaskWorker.java         |   4 +-
 .../internal/util/GridConfigurationFinder.java  |  55 +-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../internal/util/future/IgniteFutureImpl.java  |  18 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../util/nio/GridNioMessageTracker.java         |  23 +-
 .../apache/ignite/internal/visor/VisorJob.java  |   2 +
 .../internal/visor/log/VisorLogSearchTask.java  |   2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   4 +
 .../visor/query/VisorQueryCleanupTask.java      |  14 +
 .../util/VisorClusterGroupEmptyException.java   |  37 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 151 ++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 105 ++-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   3 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  74 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/spark/spark-config.xml |  46 ++
 modules/core/src/test/config/tests.properties   |   6 +-
 .../internal/ClusterGroupAbstractTest.java      | 777 ++++++++++++++++++
 .../internal/ClusterGroupHostsSelfTest.java     | 141 ++++
 .../ignite/internal/ClusterGroupSelfTest.java   | 251 ++++++
 .../internal/GridDiscoveryEventSelfTest.java    |  12 +-
 .../internal/GridProjectionAbstractTest.java    | 784 -------------------
 .../ignite/internal/GridProjectionSelfTest.java | 251 ------
 .../apache/ignite/internal/GridSelfTest.java    |   2 +-
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 +++++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ---
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++
 ...achePartitionedMultiNodeFullApiSelfTest.java |   4 +-
 .../internal/util/IgniteUtilsSelfTest.java      |  22 +
 .../GridP2PContinuousDeploymentSelfTest.java    |   2 -
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 265 ++++++-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   7 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 modules/core/src/test/resources/helloworld.gar  | Bin 6092 -> 0 bytes
 modules/core/src/test/resources/helloworld1.gar | Bin 6092 -> 0 bytes
 modules/core/src/test/resources/readme.txt      |   6 -
 modules/docker/Dockerfile                       |  55 ++
 modules/docker/README.txt                       |  11 +
 modules/docker/build_users_libs.sh              |  39 +
 modules/docker/download_ignite.sh               |  49 ++
 modules/docker/execute.sh                       |  62 ++
 modules/docker/run.sh                           |  34 +
 modules/extdata/p2p/pom.xml                     |   4 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |   2 +-
 modules/extdata/uri/META-INF/ignite.xml         |  38 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |  42 +
 .../deployment/uri/tasks/GarHelloWorldBean.java |  60 ++
 .../src/main/resources/gar-example.properties   |  18 +
 modules/extdata/uri/pom.xml                     |  62 +-
 .../deployment/uri/tasks/GarHelloWorldTask.java |  81 ++
 .../deployment/uri/tasks/gar-spring-bean.xml    |  29 +
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 .../ignite/schema/model/PojoDescriptor.java     |   2 +
 .../apache/ignite/schema/model/PojoField.java   |   1 +
 .../parser/dialect/OracleMetadataDialect.java   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |  50 +-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 +
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |  16 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |  13 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../commands/cache/VisorCacheCommand.scala      |   7 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  14 +-
 scripts/git-patch-prop.sh                       |   2 +-
 116 files changed, 3424 insertions(+), 1498 deletions(-)
----------------------------------------------------------------------


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


[21/50] incubator-ignite git commit: # IGNITE-591 Deprecate CacheConfiguration.transactionLookupClassName, create TransactionConfiguration.txLookupClassName instead.

Posted by sb...@apache.org.
# IGNITE-591 Deprecate CacheConfiguration.transactionLookupClassName, create TransactionConfiguration.txLookupClassName instead.


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

Branch: refs/heads/ignite-1085
Commit: 006209c6f4dff212c3aa7b48acb3bca9b303d5f8
Parents: b84fdaa
Author: sevdokimov <se...@jetbrains.com>
Authored: Sat Jul 4 17:09:48 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Sat Jul 4 17:09:48 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   7 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +--
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 --
 .../loadtests/hashmap/GridCacheTestContext.java |   5 +-
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   | 167 +++++++++++--------
 .../processors/cache/GridCacheJtaSelfTest.java  |  43 ++++-
 .../GridTmLookupLifecycleAwareSelfTest.java     |  30 +++-
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 16 files changed, 234 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index e2308f2..63c7269 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -967,7 +967,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Gets class name of transaction manager finder for integration for JEE app servers.
      *
      * @return Transaction manager finder.
+     * @deprecated Use {@link TransactionConfiguration#getTxManagerLookupClassName()} instead.
      */
+    @Deprecated
     public String getTransactionManagerLookupClassName() {
         return tmLookupClsName;
     }
@@ -978,7 +980,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param tmLookupClsName Name of class implementing GridCacheTmLookup interface that is used to
      *      receive JTA transaction manager.
      * @return {@code this} for chaining.
+     * @deprecated Use {@link TransactionConfiguration#setTxManagerLookupClassName(String)} instead.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setTransactionManagerLookupClassName(String tmLookupClsName) {
         this.tmLookupClsName = tmLookupClsName;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
index 9a4114b..5fed59f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java
@@ -61,6 +61,9 @@ public class TransactionConfiguration implements Serializable {
     /** Pessimistic tx log linger. */
     private int pessimisticTxLogLinger = DFLT_PESSIMISTIC_TX_LOG_LINGER;
 
+    /** Name of class implementing GridCacheTmLookup. */
+    private String tmLookupClsName;
+
     /**
      * Empty constructor.
      */
@@ -78,6 +81,7 @@ public class TransactionConfiguration implements Serializable {
         pessimisticTxLogLinger = cfg.getPessimisticTxLogLinger();
         pessimisticTxLogSize = cfg.getPessimisticTxLogSize();
         txSerEnabled = cfg.isTxSerializableEnabled();
+        tmLookupClsName = cfg.getTxManagerLookupClassName();
     }
 
     /**
@@ -202,4 +206,23 @@ public class TransactionConfiguration implements Serializable {
     public void setPessimisticTxLogLinger(int pessimisticTxLogLinger) {
         this.pessimisticTxLogLinger = pessimisticTxLogLinger;
     }
+
+    /**
+     * Gets class name of transaction manager finder for integration for JEE app servers.
+     *
+     * @return Transaction manager finder.
+     */
+    public String getTxManagerLookupClassName() {
+        return tmLookupClsName;
+    }
+
+    /**
+     * Sets look up mechanism for available {@code TransactionManager} implementation, if any.
+     *
+     * @param tmLookupClsName Name of class implementing GridCacheTmLookup interface that is used to
+     *      receive JTA transaction manager.
+     */
+    public void setTxManagerLookupClassName(String tmLookupClsName) {
+        this.tmLookupClsName = tmLookupClsName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index 755b89a..78c4722 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -186,7 +186,10 @@ public class GridCacheAttributes implements Serializable {
 
     /**
      * @return Transaction manager lookup class name.
+     * @deprecated Transaction manager lookup must be configured in 
+     *  {@link TransactionConfiguration#getTxManagerLookupClassName()}.
      */
+    @Deprecated
     public String transactionManagerLookupClassName() {
         return ccfg.getTransactionManagerLookupClassName();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 580e452..c454da9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -133,9 +133,6 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Replication manager. */
     private GridCacheDrManager drMgr;
 
-    /** JTA manager. */
-    private CacheJtaManagerAdapter jtaMgr;
-
     /** Conflict resolver manager. */
     private CacheConflictResolutionManager rslvrMgr;
 
@@ -251,7 +248,6 @@ public class GridCacheContext<K, V> implements Externalizable {
         CacheDataStructuresManager dataStructuresMgr,
         GridCacheTtlManager ttlMgr,
         GridCacheDrManager drMgr,
-        CacheJtaManagerAdapter jtaMgr,
         CacheConflictResolutionManager<K, V> rslvrMgr,
         CachePluginManager pluginMgr
     ) {
@@ -292,7 +288,6 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.dataStructuresMgr = add(dataStructuresMgr);
         this.ttlMgr = add(ttlMgr);
         this.drMgr = add(drMgr);
-        this.jtaMgr = add(jtaMgr);
         this.rslvrMgr = add(rslvrMgr);
         this.pluginMgr = add(pluginMgr);
 
@@ -1018,7 +1013,7 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @return JTA manager.
      */
     public CacheJtaManagerAdapter jta() {
-        return jtaMgr;
+        return sharedCtx.jta();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/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 8124594..1acb351 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
@@ -495,7 +495,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cleanup(cfg, cfg.getEvictionPolicy(), false);
         cleanup(cfg, cfg.getAffinity(), false);
         cleanup(cfg, cfg.getAffinityMapper(), false);
-        cleanup(cfg, cctx.jta().tmLookup(), false);
         cleanup(cfg, cctx.store().configuredStore(), false);
 
         NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
@@ -1001,8 +1000,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         ctx.kernalContext().query().onCacheStop(ctx);
         ctx.kernalContext().continuous().onCacheStop(ctx);
 
-        U.stopLifecycleAware(log, lifecycleAwares(cache.configuration(), ctx.jta().tmLookup(),
-            ctx.store().configuredStore()));
+        U.stopLifecycleAware(log, lifecycleAwares(cache.configuration(), ctx.store().configuredStore()));
 
         if (log.isInfoEnabled())
             log.info("Stopped cache: " + cache.name());
@@ -1120,9 +1118,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         pluginMgr.validate();
 
-        CacheJtaManagerAdapter jta = JTA.create(cfg.getTransactionManagerLookupClassName() == null);
-
-        jta.createTmLookup(cfg);
+        sharedCtx.jta().registerCache(cfg);
 
         // Skip suggestions for internal caches.
         if (cacheType.userCache())
@@ -1130,8 +1126,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         Collection<Object> toPrepare = new ArrayList<>();
 
-        toPrepare.add(jta.tmLookup());
-
         if (cfgStore instanceof GridCacheLoaderWriterStore) {
             toPrepare.add(((GridCacheLoaderWriterStore)cfgStore).loader());
             toPrepare.add(((GridCacheLoaderWriterStore)cfgStore).writer());
@@ -1141,7 +1135,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         prepare(cfg, toPrepare);
 
-        U.startLifecycleAware(lifecycleAwares(cfg, jta.tmLookup(), cfgStore));
+        U.startLifecycleAware(lifecycleAwares(cfg, cfgStore));
 
         GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
         GridCacheEventManager evtMgr = new GridCacheEventManager();
@@ -1181,7 +1175,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             dataStructuresMgr,
             ttlMgr,
             drMgr,
-            jta,
             rslvrMgr,
             pluginMgr
         );
@@ -1311,7 +1304,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 dataStructuresMgr,
                 ttlMgr,
                 drMgr,
-                jta,
                 rslvrMgr,
                 pluginMgr
             );
@@ -1632,7 +1624,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("unchecked")
     private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx,
-        Collection<CacheStoreSessionListener> storeSesLsnrs) {
+        Collection<CacheStoreSessionListener> storeSesLsnrs) throws IgniteCheckedException {
         IgniteTxManager tm = new IgniteTxManager();
         GridCacheMvccManager mvccMgr = new GridCacheMvccManager();
         GridCacheVersionManager verMgr = new GridCacheVersionManager();
@@ -1640,6 +1632,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCachePartitionExchangeManager exchMgr = new GridCachePartitionExchangeManager();
         GridCacheIoManager ioMgr = new GridCacheIoManager();
 
+        CacheJtaManagerAdapter jta = JTA.createOptional();
+        
         return new GridCacheSharedContext(
             kernalCtx,
             tm,
@@ -1648,7 +1642,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             depMgr,
             exchMgr,
             ioMgr,
-            storeSesLsnrs
+            storeSesLsnrs,
+            jta
         );
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 1071ef2..7f4daff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.jta.*;
 import org.apache.ignite.internal.processors.cache.store.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -54,6 +55,9 @@ public class GridCacheSharedContext<K, V> {
     /** Cache transaction manager. */
     private IgniteTxManager txMgr;
 
+    /** JTA manager. */
+    private CacheJtaManagerAdapter jtaMgr;
+
     /** Partition exchange manager. */
     private GridCachePartitionExchangeManager<K, V> exchMgr;
 
@@ -94,12 +98,14 @@ public class GridCacheSharedContext<K, V> {
         GridCacheDeploymentManager<K, V> depMgr,
         GridCachePartitionExchangeManager<K, V> exchMgr,
         GridCacheIoManager ioMgr,
-        Collection<CacheStoreSessionListener> storeSesLsnrs
+        Collection<CacheStoreSessionListener> storeSesLsnrs,
+        CacheJtaManagerAdapter jtaMgr
     ) {
         this.kernalCtx = kernalCtx;
         this.mvccMgr = add(mvccMgr);
         this.verMgr = add(verMgr);
         this.txMgr = add(txMgr);
+        this.jtaMgr = add(jtaMgr);
         this.depMgr = add(depMgr);
         this.exchMgr = add(exchMgr);
         this.ioMgr = add(ioMgr);
@@ -281,6 +287,13 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @return JTA manager.
+     */
+    public CacheJtaManagerAdapter jta() {
+        return jtaMgr;
+    }
+
+    /**
      * @return Exchange manager.
      */
     public GridCachePartitionExchangeManager<K, V> exchange() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManagerAdapter.java
index ef9204d..85fa4b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManagerAdapter.java
@@ -25,15 +25,7 @@ import org.jetbrains.annotations.*;
 /**
  * Provides possibility to integrate cache transactions with JTA.
  */
-public abstract class CacheJtaManagerAdapter extends GridCacheManagerAdapter {
-    /**
-     * Creates transaction manager finder.
-     *
-     * @param ccfg Cache configuration.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract void createTmLookup(CacheConfiguration ccfg) throws IgniteCheckedException;
-
+public abstract class CacheJtaManagerAdapter extends GridCacheSharedManagerAdapter {
     /**
      * Checks if cache is working in JTA transaction and enlist cache as XAResource if necessary.
      *
@@ -42,6 +34,13 @@ public abstract class CacheJtaManagerAdapter extends GridCacheManagerAdapter {
     public abstract void checkJta() throws IgniteCheckedException;
 
     /**
+     * @param cfg Cache configuration.
+     * @throws IgniteCheckedException If {@link CacheConfiguration#getTransactionManagerLookupClassName()} is incompatible with
+     *     another caches or {@link TransactionConfiguration#getTxManagerLookupClassName()}.
+     */
+    public abstract void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException;
+
+    /**
      * Gets transaction manager finder. Returns Object to avoid dependency on JTA library.
      *
      * @return Transaction manager finder.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheNoopJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheNoopJtaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheNoopJtaManager.java
index 451357f..60e4102 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheNoopJtaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheNoopJtaManager.java
@@ -31,7 +31,7 @@ public class CacheNoopJtaManager extends CacheJtaManagerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void createTmLookup(CacheConfiguration ccfg) throws IgniteCheckedException {
+    @Override public void registerCache(CacheConfiguration<?, ?> cfg) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index 4097c4f..b983039 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -62,9 +62,6 @@ public class VisorCacheConfiguration implements Serializable {
     /** Start size. */
     private int startSize;
 
-    /** Name of class implementing GridCacheTmLookup. */
-    private String tmLookupClsName;
-
     /** Off-heap max memory. */
     private long offHeapMaxMemory;
 
@@ -134,7 +131,6 @@ public class VisorCacheConfiguration implements Serializable {
         swapEnabled = ccfg.isSwapEnabled();
         invalidate = ccfg.isInvalidate();
         startSize = ccfg.getStartSize();
-        tmLookupClsName = ccfg.getTransactionManagerLookupClassName();
         offHeapMaxMemory = ccfg.getOffHeapMaxMemory();
         maxConcurrentAsyncOps = ccfg.getMaxConcurrentAsyncOperations();
         memoryMode = ccfg.getMemoryMode();
@@ -222,13 +218,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @return Name of class implementing GridCacheTmLookup.
-     */
-    @Nullable public String transactionManagerLookupClassName() {
-        return tmLookupClsName;
-    }
-
-    /**
      * @return Off-heap max memory.
      */
     public long offsetHeapMaxMemory() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 77d3905..f0d1af8 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -19,6 +19,7 @@ package org.apache.ignite.loadtests.hashmap;
 
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.datastructures.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
@@ -55,7 +56,8 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheDeploymentManager<K, V>(),
                 new GridCachePartitionExchangeManager<K, V>(),
                 new GridCacheIoManager(),
-                null
+                null,
+                new CacheNoopJtaManager()
             ),
             defaultCacheConfiguration(),
             CacheType.USER,
@@ -71,7 +73,6 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             new CacheDataStructuresManager(),
             new GridCacheTtlManager(),
             new GridOsCacheDrManager(),
-            new CacheNoopJtaManager(),
             new CacheOsConflictResolutionManager<K, V>(),
             new CachePluginManager(ctx, new CacheConfiguration())
         );

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java
index 6a1fd2b..0e2526e 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/HibernateTransactionalDataRegion.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cache.hibernate;
 
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.hibernate.cache.*;
 import org.hibernate.cache.spi.*;
@@ -83,9 +84,14 @@ public class HibernateTransactionalDataRegion extends HibernateRegion implements
                     throw new CacheException("Hibernate TRANSACTIONAL access strategy must have Ignite cache with " +
                         "'TRANSACTIONAL' atomicity mode: " + cache.name());
 
-                if (cache.configuration().getTransactionManagerLookupClassName() == null)
-                    throw new CacheException("Hibernate TRANSACTIONAL access strategy must have Ignite cache with " +
-                        "TransactionManagerLookup configured: " + cache.name());
+                if (cache.configuration().getTransactionManagerLookupClassName() == null) {
+                    TransactionConfiguration txCfg = ignite.configuration().getTransactionConfiguration();
+                    
+                    if (txCfg == null || txCfg.getTxManagerLookupClassName() == null)
+                        throw new CacheException("Hibernate TRANSACTIONAL access strategy must have Ignite with " +
+                            "TransactionManagerLookup configured (see IgniteConfiguration." +
+                            "getTransactionConfiguration().getTxManagerLookupClassName()): " + cache.name());
+                }
 
                 return new HibernateTransactionalAccessStrategy(ignite, cache);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java b/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
index c909c7c..5a38576 100644
--- a/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
+++ b/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cache.jta;
 
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.TransactionConfiguration;
 import org.jetbrains.annotations.*;
 
 import javax.transaction.*;
@@ -25,7 +26,7 @@ import javax.transaction.*;
 /**
  * Allows grid to use different transactional systems. Implement this interface
  * to look up native transaction manager within your environment. Transaction
- * manager lookup is configured via {@link org.apache.ignite.configuration.CacheConfiguration#getTransactionManagerLookupClassName()}
+ * manager lookup is configured via {@link TransactionConfiguration#getTxManagerLookupClassName()}
  * method.
  * <p>
  * The following implementations are provided out of the box:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index 2a0d46d..09d856b 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -21,37 +21,60 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.jta.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
+import org.apache.ignite.lifecycle.*;
 import org.jetbrains.annotations.*;
 
 import javax.transaction.*;
-import java.lang.ref.*;
-import java.util.*;
+import java.util.concurrent.atomic.*;
 
 /**
  * Implementation of {@link CacheJtaManagerAdapter}.
  */
 public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** */
-    private static final Map<TransactionManager, ThreadLocal<WeakReference<GridCacheXAResource>>> threadLocals =
-            new WeakHashMap<>();
+    private final ThreadLocal<GridCacheXAResource> xaRsrc = new ThreadLocal<>();
 
     /** */
     private volatile TransactionManager jtaTm;
 
     /** */
-    private ThreadLocal<WeakReference<GridCacheXAResource>> xaRsrc;
+    private final AtomicReference<CacheTmLookup> tmLookupRef = new AtomicReference<>();
 
-    /** */
-    private CacheTmLookup tmLookup;
+    /** {@inheritDoc} */
+    @Override protected void start0() throws IgniteCheckedException {
+        super.start0();
+
+        if (cctx.txConfig() != null) {
+            String txLookupClsName = cctx.txConfig().getTxManagerLookupClassName();
+            
+            if (txLookupClsName != null)
+                tmLookupRef.set(createTmLookup(txLookupClsName));
+        }
+    }
 
     /** {@inheritDoc} */
-    @Override public void createTmLookup(CacheConfiguration ccfg) throws IgniteCheckedException {
-        assert ccfg.getTransactionManagerLookupClassName() != null;
+    @Override protected void stop0(boolean cancel) {
+        CacheTmLookup tmLookup = tmLookupRef.get();
+        
+        if (tmLookup instanceof LifecycleAware)
+            ((LifecycleAware)tmLookup).stop();
+    }
 
+    /**
+     * @throws IgniteCheckedException
+     */
+    private CacheTmLookup createTmLookup(String tmLookupClsName) throws IgniteCheckedException {
         try {
-            Class<?> cls = Class.forName(ccfg.getTransactionManagerLookupClassName());
+            Class<?> cls = Class.forName(tmLookupClsName);
+
+            CacheTmLookup res = (CacheTmLookup)cls.newInstance();
 
-            tmLookup = (CacheTmLookup)cls.newInstance();
+            cctx.kernalContext().resource().injectGeneric(res);
+            
+            if (res instanceof LifecycleAware)
+                ((LifecycleAware)res).start();
+            
+            return res;
         }
         catch (Exception e) {
             throw new IgniteCheckedException("Failed to instantiate transaction manager lookup.", e);
@@ -60,82 +83,90 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
 
     /** {@inheritDoc} */
     @Override public void checkJta() throws IgniteCheckedException {
-        TransactionManager jtaTm = this.jtaTm;
-
         if (jtaTm == null) {
             try {
-                jtaTm = tmLookup.getTm();
-
-                if (jtaTm == null)
+                CacheTmLookup tmLookup = tmLookupRef.get();
+                
+                if (tmLookup == null)
                     return;
-
-                synchronized (threadLocals) {
-                    if (this.jtaTm != null)
-                        jtaTm = this.jtaTm;
-                    else {
-                        xaRsrc = threadLocals.get(jtaTm);
-
-                        if (xaRsrc == null) {
-                            xaRsrc = new ThreadLocal<>();
-
-                            threadLocals.put(jtaTm, xaRsrc);
-                        }
-
-                        this.jtaTm = jtaTm;
-                    }
-                }
+                
+                jtaTm = tmLookup.getTm();
             }
             catch (Exception e) {
                 throw new IgniteCheckedException("Failed to get transaction manager: " + e, e);
             }
         }
 
-        WeakReference<GridCacheXAResource> rsrcRef = xaRsrc.get();
-
-        GridCacheXAResource rsrc = rsrcRef == null ? null : rsrcRef.get();
-
-        if (rsrc == null || rsrc.isFinished()) {
-            try {
-                Transaction jtaTx = jtaTm.getTransaction();
-
-                if (jtaTx != null) {
-                    IgniteInternalTx tx = cctx.tm().userTx();
-
-                    if (tx == null) {
-                        TransactionConfiguration tCfg = cctx.kernalContext().config()
-                            .getTransactionConfiguration();
-
-                        tx = cctx.tm().newTx(
-                            /*implicit*/false,
-                            /*implicit single*/false,
-                            null,
-                            tCfg.getDefaultTxConcurrency(),
-                            tCfg.getDefaultTxIsolation(),
-                            tCfg.getDefaultTxTimeout(),
-                            /*store enabled*/true,
-                            /*tx size*/0
-                        );
-                    }
+        if (jtaTm != null) {
+            GridCacheXAResource rsrc = xaRsrc.get();
+
+            if (rsrc == null || rsrc.isFinished()) {
+                try {
+                    Transaction jtaTx = jtaTm.getTransaction();
+
+                    if (jtaTx != null) {
+                        IgniteInternalTx tx = cctx.tm().userTx();
+
+                        if (tx == null) {
+                            TransactionConfiguration tCfg = cctx.kernalContext().config()
+                                .getTransactionConfiguration();
+
+                            tx = cctx.tm().newTx(
+                                /*implicit*/false,
+                                /*implicit single*/false,
+                                null,
+                                tCfg.getDefaultTxConcurrency(),
+                                tCfg.getDefaultTxIsolation(),
+                                tCfg.getDefaultTxTimeout(),
+                                /*store enabled*/true,
+                                /*tx size*/0
+                            );
+                        }
 
-                    rsrc = new GridCacheXAResource(tx, cctx.kernalContext());
+                        rsrc = new GridCacheXAResource(tx, cctx.kernalContext());
 
-                    if (!jtaTx.enlistResource(rsrc))
-                        throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");
+                        if (!jtaTx.enlistResource(rsrc))
+                            throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");
 
-                    xaRsrc.set(new WeakReference<>(rsrc));
+                        xaRsrc.set(rsrc);
+                    }
+                }
+                catch (SystemException e) {
+                    throw new IgniteCheckedException("Failed to obtain JTA transaction.", e);
+                }
+                catch (RollbackException e) {
+                    throw new IgniteCheckedException("Failed to enlist XAResource to JTA transaction.", e);
                 }
             }
-            catch (SystemException e) {
-                throw new IgniteCheckedException("Failed to obtain JTA transaction.", e);
-            }
-            catch (RollbackException e) {
-                throw new IgniteCheckedException("Failed to enlist XAResource to JTA transaction.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void registerCache(CacheConfiguration<?, ?> cfg) throws IgniteCheckedException {
+        String cacheLookupClsName = cfg.getTransactionManagerLookupClassName();
+        
+        if (cacheLookupClsName != null) {
+            CacheTmLookup tmLookup = tmLookupRef.get();
+            
+            if (tmLookup == null) {
+                tmLookup = createTmLookup(cacheLookupClsName);
+                
+                if (tmLookupRef.compareAndSet(null, tmLookup))
+                    return;
+                
+                tmLookup = tmLookupRef.get();
             }
+
+            if (!cacheLookupClsName.equals(tmLookup.getClass().getName()))
+                throw new IgniteCheckedException("Failed to start cache with CacheTmLookup that specified in cache " +
+                    "configuration, because node uses another CacheTmLookup [cache" + cfg.getName() +
+                    ", tmLookupClassName=" + cacheLookupClsName + ", tmLookupUsedByNode=" 
+                    + tmLookup.getClass().getName() + ']');
         }
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public Object tmLookup() {
-        return tmLookup;
+        return tmLookupRef.get();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index e1c8a77..29836ef 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -22,11 +22,14 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.jta.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.Transaction;
 import org.objectweb.jotm.*;
 
 import javax.transaction.*;
 
+import java.util.concurrent.*;
+
 import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.transactions.TransactionState.*;
 
@@ -65,18 +68,11 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setTransactionManagerLookupClassName(TestTmLookup.class.getName());
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.getTransactionConfiguration().setTxManagerLookupClassName(TestTmLookup.class.getName());
+        
         CacheConfiguration cfg1 = cacheConfiguration(gridName);
 
         CacheConfiguration cfg2 = cacheConfiguration(gridName);
@@ -100,6 +96,17 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
     }
 
     /**
+     *
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class TestTmLookup2 implements CacheTmLookup {
+        /** {@inheritDoc} */
+        @Override public TransactionManager getTm() {
+            return null;
+        }
+    }
+
+    /**
      * JUnit.
      *
      * @throws Exception If failed.
@@ -185,4 +192,22 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
         assertEquals(1, (int)cache1.get("key"));
         assertEquals(1, (int)cache2.get("key"));
     }
+    
+    public void testUncompatibleTmLookup() {
+        final IgniteEx ignite = grid(0);
+        
+        final CacheConfiguration cacheCfg = new CacheConfiguration();
+        
+        cacheCfg.setName("Foo");
+        cacheCfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        cacheCfg.setTransactionManagerLookupClassName(TestTmLookup2.class.getName());
+        
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws IgniteException {
+                ignite.createCache(cacheCfg);
+                
+                return null;
+            }
+        }, IgniteException.class, null);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
index 5465a91..261765b 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.jta.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
@@ -41,15 +42,24 @@ public class GridTmLookupLifecycleAwareSelfTest extends GridAbstractLifecycleAwa
     /** */
     private boolean near;
 
+    /** */
+    private boolean configureGlobalTmLookup;
+    
     /**
      */
     @SuppressWarnings("PublicInnerClass")
     public static class TestTxLookup extends GridAbstractLifecycleAwareSelfTest.TestLifecycleAware
         implements CacheTmLookup {
-        /**
-         */
-        public TestTxLookup() {
-            super(CACHE_NAME);
+
+        /** */
+        @IgniteInstanceResource
+        Ignite ignite;
+        
+        /** {@inheritDoc} */
+        @Override public void start() {
+            super.start();
+            
+            assertNotNull(ignite);
         }
 
         /** {@inheritDoc} */
@@ -74,7 +84,10 @@ public class GridTmLookupLifecycleAwareSelfTest extends GridAbstractLifecycleAwa
 
         ccfg.setName(CACHE_NAME);
 
-        ccfg.setTransactionManagerLookupClassName(TestTxLookup.class.getName());
+        if (configureGlobalTmLookup)
+            cfg.getTransactionConfiguration().setTxManagerLookupClassName(TestTxLookup.class.getName());
+        else
+            ccfg.setTransactionManagerLookupClassName(TestTxLookup.class.getName());
 
         cfg.setCacheConfiguration(ccfg);
 
@@ -99,4 +112,11 @@ public class GridTmLookupLifecycleAwareSelfTest extends GridAbstractLifecycleAwa
             super.testLifecycleAware();
         }
     }
+
+    /** {@inheritDoc} */
+    public void testLifecycleAwareGlobal() throws Exception {
+        configureGlobalTmLookup = true;
+
+        super.testLifecycleAware();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/006209c6/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 4c005af..90c2de0 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -833,8 +833,6 @@ object VisorCacheCommand {
         cacheT += ("Invalidate", bool2Str(cfg.invalidate()))
         cacheT += ("Start Size", cfg.startSize())
 
-        cacheT += ("Transaction Manager Lookup", safe(cfg.transactionManagerLookupClassName()))
-
         cacheT += ("Affinity Function", safe(affinityCfg.function()))
         cacheT += ("Affinity Backups", affinityCfg.partitionedBackups())
         cacheT += ("Affinity Partitions", safe(affinityCfg.partitions()))



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

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


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

Branch: refs/heads/ignite-1085
Commit: fa5cb918dcb0fe4253332fd9be859803ad014ddb
Parents: 459d702 4c9d8c2
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:39:36 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:39:36 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheIoManager.java    |   8 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../distributed/near/GridNearGetFuture.java     |   4 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 -
 .../cache/CacheFutureExceptionSelfTest.java     | 161 +++++++--------
 .../loadtests/hashmap/GridCacheTestContext.java |   4 +-
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 -
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   |  72 ++++++-
 .../cache/jta/GridCacheXAResource.java          |  16 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |  52 +++--
 .../GridTmLookupLifecycleAwareSelfTest.java     |  29 ++-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/kafka/pom.xml                           |  11 -
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 .../config/benchmark-index-put.properties       |  64 ------
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 .../cache/IgnitePutIndex1Benchmark.java         |  42 ----
 .../cache/IgnitePutIndex2Benchmark.java         |  42 ----
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |   2 +-
 .../ignite/yardstick/cache/model/Person1.java   |  29 +--
 .../ignite/yardstick/cache/model/Person2.java   |  45 +----
 .../ignite/yardstick/cache/model/Person8.java   | 155 +-------------
 34 files changed, 661 insertions(+), 558 deletions(-)
----------------------------------------------------------------------



[15/50] incubator-ignite git commit: # IGNITE-591 Don't use static ThreadLocal.

Posted by sb...@apache.org.
# IGNITE-591 Don't use static ThreadLocal.


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

Branch: refs/heads/ignite-1085
Commit: 53143355dc713f2af3308f5ceef9fcc6e3d3d2a1
Parents: bac6f48
Author: sevdokimov <se...@gridgain.com>
Authored: Thu Jul 2 16:38:10 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu Jul 2 16:38:10 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/jta/CacheJtaManager.java   | 122 ++++++++++++-------
 1 file changed, 77 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/53143355/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index 2a078dc..c5a94bc 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -17,23 +17,34 @@
 
 package org.apache.ignite.internal.processors.cache.jta;
 
-import org.apache.ignite.*;
-import org.apache.ignite.cache.jta.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
-import org.jetbrains.annotations.*;
-
-import javax.transaction.*;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.jta.CacheTmLookup;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.jetbrains.annotations.Nullable;
+
+import javax.transaction.RollbackException;
+import javax.transaction.SystemException;
+import javax.transaction.Transaction;
+import javax.transaction.TransactionManager;
+import java.lang.ref.WeakReference;
+import java.util.Map;
+import java.util.WeakHashMap;
 
 /**
  * Implementation of {@link CacheJtaManagerAdapter}.
  */
 public class CacheJtaManager extends CacheJtaManagerAdapter {
     /** */
-    private final static ThreadLocal<GridCacheXAResource> xaRsrc = new ThreadLocal<>();
+    private static final Map<TransactionManager, ThreadLocal<WeakReference<GridCacheXAResource>>> threadLocals =
+            new WeakHashMap<>();
+
+    /** */
+    private volatile TransactionManager jtaTm;
 
     /** */
-    private TransactionManager jtaTm;
+    private ThreadLocal<WeakReference<GridCacheXAResource>> xaRsrc;
 
     /** */
     private CacheTmLookup tmLookup;
@@ -54,55 +65,76 @@ public class CacheJtaManager extends CacheJtaManagerAdapter {
 
     /** {@inheritDoc} */
     @Override public void checkJta() throws IgniteCheckedException {
+        TransactionManager jtaTm = this.jtaTm;
+
         if (jtaTm == null) {
             try {
                 jtaTm = tmLookup.getTm();
+
+                if (jtaTm == null)
+                    return;
+
+                synchronized (threadLocals) {
+                    if (this.jtaTm != null)
+                        jtaTm = this.jtaTm;
+                    else {
+                        xaRsrc = threadLocals.get(jtaTm);
+
+                        if (xaRsrc == null) {
+                            xaRsrc = new ThreadLocal<>();
+
+                            threadLocals.put(jtaTm, xaRsrc);
+                        }
+
+                        this.jtaTm = jtaTm;
+                    }
+                }
             }
             catch (Exception e) {
                 throw new IgniteCheckedException("Failed to get transaction manager: " + e, e);
             }
         }
 
-        if (jtaTm != null) {
-            GridCacheXAResource rsrc = xaRsrc.get();
-
-            if (rsrc == null || rsrc.isFinished()) {
-                try {
-                    Transaction jtaTx = jtaTm.getTransaction();
-
-                    if (jtaTx != null) {
-                        IgniteInternalTx tx = cctx.tm().userTx();
-
-                        if (tx == null) {
-                            TransactionConfiguration tCfg = cctx.kernalContext().config()
-                                .getTransactionConfiguration();
-
-                            tx = cctx.tm().newTx(
-                                /*implicit*/false,
-                                /*implicit single*/false,
-                                null,
-                                tCfg.getDefaultTxConcurrency(),
-                                tCfg.getDefaultTxIsolation(),
-                                tCfg.getDefaultTxTimeout(),
-                                /*store enabled*/true,
-                                /*tx size*/0
-                            );
-                        }
-
-                        rsrc = new GridCacheXAResource(tx, cctx.kernalContext());
+        WeakReference<GridCacheXAResource> rsrcRef = xaRsrc.get();
 
-                        if (!jtaTx.enlistResource(rsrc))
-                            throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");
+        GridCacheXAResource rsrc = rsrcRef == null ? null : rsrcRef.get();
 
-                        xaRsrc.set(rsrc);
+        if (rsrc == null || rsrc.isFinished()) {
+            try {
+                Transaction jtaTx = jtaTm.getTransaction();
+
+                if (jtaTx != null) {
+                    IgniteInternalTx tx = cctx.tm().userTx();
+
+                    if (tx == null) {
+                        TransactionConfiguration tCfg = cctx.kernalContext().config()
+                            .getTransactionConfiguration();
+
+                        tx = cctx.tm().newTx(
+                            /*implicit*/false,
+                            /*implicit single*/false,
+                            null,
+                            tCfg.getDefaultTxConcurrency(),
+                            tCfg.getDefaultTxIsolation(),
+                            tCfg.getDefaultTxTimeout(),
+                            /*store enabled*/true,
+                            /*tx size*/0
+                        );
                     }
+
+                    rsrc = new GridCacheXAResource(tx, cctx.kernalContext());
+
+                    if (!jtaTx.enlistResource(rsrc))
+                        throw new IgniteCheckedException("Failed to enlist XA resource to JTA user transaction.");
+
+                    xaRsrc.set(new WeakReference<>(rsrc));
                 }
-                catch (SystemException e) {
-                    throw new IgniteCheckedException("Failed to obtain JTA transaction.", e);
-                }
-                catch (RollbackException e) {
-                    throw new IgniteCheckedException("Failed to enlist XAResource to JTA transaction.", e);
-                }
+            }
+            catch (SystemException e) {
+                throw new IgniteCheckedException("Failed to obtain JTA transaction.", e);
+            }
+            catch (RollbackException e) {
+                throw new IgniteCheckedException("Failed to enlist XAResource to JTA transaction.", e);
             }
         }
     }


[38/50] incubator-ignite git commit: # Fixed unmarshalling error handling for cache 'get'

Posted by sb...@apache.org.
# Fixed unmarshalling error handling for cache 'get'


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

Branch: refs/heads/ignite-1085
Commit: 6d6ec778b43d730dd19001011aaaa49ec86c5d20
Parents: 94a42a4
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 8 15:08:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 8 15:08:33 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  8 ++-
 .../distributed/near/GridNearGetFuture.java     |  4 +-
 .../cache/CacheFutureExceptionSelfTest.java     | 72 +++++++++++++++++---
 3 files changed, 71 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d6ec778/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 0707096..29e3551 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
@@ -445,8 +445,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             case 50: {
                 GridNearGetResponse res = (GridNearGetResponse)msg;
 
-                GridPartitionedGetFuture fut = (GridPartitionedGetFuture)ctx.mvcc().future(
-                    res.version(), res.futureId());
+                GridCacheFuture fut = ctx.mvcc().future(res.version(), res.futureId());
 
                 if (fut == null) {
                     if (log.isDebugEnabled())
@@ -457,7 +456,10 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(res.classError());
 
-                fut.onResult(nodeId, res);
+                if (fut instanceof GridNearGetFuture)
+                    ((GridNearGetFuture)fut).onResult(nodeId, res);
+                else
+                    ((GridPartitionedGetFuture)fut).onResult(nodeId, res);
             }
 
             break;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d6ec778/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 74438bb..58f6fe5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -223,7 +223,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param nodeId Sender.
      * @param res Result.
      */
-    void onResult(UUID nodeId, GridNearGetResponse res) {
+    public void onResult(UUID nodeId, GridNearGetResponse res) {
         for (IgniteInternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
@@ -649,7 +649,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     if (log.isDebugEnabled())
                         log.debug("Got removed entry while processing get response (will not retry).");
                 }
-                catch (IgniteCheckedException e) {
+                catch (Exception e) {
                     // Fail.
                     onDone(e);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d6ec778/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
index 34d2daa..372c859 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
@@ -20,6 +20,9 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.*;
@@ -32,31 +35,82 @@ import static java.util.concurrent.TimeUnit.*;
  * Cache future self test.
  */
 public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static volatile boolean fail;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = new IgniteConfiguration();
+
         cfg.setGridName(gridName);
 
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(spi);
+
         if (gridName.equals(getTestGridName(1)))
             cfg.setClientMode(true);
 
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testAsyncCacheFuture() throws Exception {
-        Ignite srv = startGrid(0);
+        startGrid(0);
+
+        startGrid(1);
+
+        testGet(false, false);
+
+        testGet(false, true);
+
+        testGet(true, false);
+
+        testGet(true, true);
+    }
+
+    /**
+     * @param nearCache If {@code true} creates near cache on client.
+     * @param cpyOnRead Cache copy on read flag.
+     * @throws Exception If failed.
+     */
+    private void testGet(boolean nearCache, boolean cpyOnRead) throws Exception {
+        fail = false;
+
+        Ignite srv = grid(0);
+
+        Ignite client = grid(1);
+
+        final String cacheName = nearCache ? ("NEAR-CACHE-" + cpyOnRead) : ("CACHE-" + cpyOnRead);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setCopyOnRead(cpyOnRead);
+
+        ccfg.setName(cacheName);
+
+        IgniteCache<Object, Object> cache = srv.createCache(ccfg);
 
-        IgniteCache<String, NotSerializableClass> cache = srv.getOrCreateCache("CACHE");
         cache.put("key", new NotSerializableClass());
 
-        Ignite client = startGrid(1);
+        IgniteCache<Object, Object> clientCache = nearCache ? client.createNearCache(cacheName,
+            new NearCacheConfiguration<>()) : client.cache(cacheName);
 
-        IgniteCache<String, NotSerializableClass> asyncCache = client.<String, NotSerializableClass>cache("CACHE").withAsync();
+        IgniteCache<Object, Object> asyncCache = clientCache.withAsync();
 
-        System.setProperty("FAIL", "true");
+        fail = true;
 
         asyncCache.get("key");
 
@@ -79,7 +133,9 @@ public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        assertTrue(futLatch.await(60, SECONDS));
+        assertTrue(futLatch.await(5, SECONDS));
+
+        srv.destroyCache(cache.getName());
     }
 
     /**
@@ -93,10 +149,10 @@ public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc}*/
         private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-            if (System.getProperty("FAIL") != null)
+            if (fail)
                 throw new RuntimeException("Deserialization failed.");
 
             in.readObject();
         }
     }
-}	
\ No newline at end of file
+}
\ No newline at end of file


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

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


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

Branch: refs/heads/ignite-1085
Commit: 7cf9d76b8323162ecae69b26cc3d5fd12320b7a3
Parents: 431a62e 415264e
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 19 14:37:41 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 19 14:37:41 2015 -0700

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../GridCachePartitionExchangeManager.java      |   1 +
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  71 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 .../core/src/main/resources/ignite.properties   |   2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  44 ++++-
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 59 files changed, 545 insertions(+), 73 deletions(-)
----------------------------------------------------------------------



[18/50] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by sb...@apache.org.
IGNITE-1026 - Count down latch 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/d27453fd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d27453fd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d27453fd

Branch: refs/heads/ignite-1085
Commit: d27453fdcdea4612a7ba4d0ef6b0168eb5569ed0
Parents: d38ad8c
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jul 2 17:52:56 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jul 2 17:52:56 2015 -0700

----------------------------------------------------------------------
 .../IgniteCountDownLatchAbstractSelfTest.java           | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d27453fd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
index 80e6123..01ffaf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
@@ -19,9 +19,9 @@ package org.apache.ignite.internal.processors.cache.datastructures;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
 
@@ -133,7 +133,13 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
      *
      * @throws Exception If failed.
      */
-    protected void checkRemovedLatch(IgniteCountDownLatch latch) throws Exception {
+    protected void checkRemovedLatch(final IgniteCountDownLatch latch) throws Exception {
+        assert GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                return latch.removed();
+            }
+        }, 5000);
+
         assert latch.removed();
 
         assert latch.count() == 0;
@@ -177,7 +183,7 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
      */
     private void checkAwait() throws Exception {
         // Check only 'false' cases here. Successful await is tested over the grid.
-        IgniteCountDownLatch latch = createLatch("await", 5, true);
+        IgniteCountDownLatch latch = createLatch("await", 5, false);
 
         assert !latch.await(10);
         assert !latch.await(10, MILLISECONDS);


[07/50] incubator-ignite git commit: # IGNITE-1026 - Dynamic cache start fix

Posted by sb...@apache.org.
# IGNITE-1026 - Dynamic cache start 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/9adc180b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9adc180b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9adc180b

Branch: refs/heads/ignite-1085
Commit: 9adc180bcf1320bfb92d387e64c82a2f5b1029b6
Parents: 08ea4cc
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 19 15:50:30 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 19 15:50:30 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java    | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9adc180b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 3ee260d..6c7c511 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -77,7 +77,6 @@ import javax.management.*;
 import java.io.*;
 import java.lang.management.*;
 import java.lang.reflect.*;
-import java.security.*;
 import java.text.*;
 import java.util.*;
 import java.util.concurrent.*;
@@ -2248,7 +2247,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            return ctx.cache().publicJCache(name, true);
+            return ctx.cache().publicJCache(name, false);
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);


[19/50] incubator-ignite git commit: IGNITE-591 Fix imports.

Posted by sb...@apache.org.
IGNITE-591 Fix imports.


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

Branch: refs/heads/ignite-1085
Commit: bb8dcf9ce0290a4016c13184aef9796c7a2d7bde
Parents: 5314335
Author: sevdokimov <se...@jetbrains.com>
Authored: Fri Jul 3 11:09:43 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Fri Jul 3 11:09:43 2015 +0300

----------------------------------------------------------------------
 .../hibernate/HibernateL2CacheSelfTest.java     | 45 +++++++++-----------
 .../HibernateL2CacheTransactionalSelfTest.java  | 33 +++++++-------
 .../processors/cache/jta/CacheJtaManager.java   | 23 ++++------
 .../processors/cache/GridCacheJtaSelfTest.java  | 21 ++++-----
 4 files changed, 55 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb8dcf9c/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
index fe34ea8..91d5c26 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
@@ -17,40 +17,37 @@
 
 package org.apache.ignite.cache.hibernate;
 
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
 import org.hibernate.*;
 import org.hibernate.Query;
-import org.hibernate.annotations.NaturalId;
-import org.hibernate.annotations.NaturalIdCache;
-import org.hibernate.cache.spi.GeneralDataRegion;
-import org.hibernate.cache.spi.TransactionalDataRegion;
+import org.hibernate.annotations.*;
+import org.hibernate.cache.spi.*;
 import org.hibernate.cache.spi.access.AccessType;
-import org.hibernate.cfg.Configuration;
-import org.hibernate.exception.ConstraintViolationException;
-import org.hibernate.service.ServiceRegistryBuilder;
-import org.hibernate.stat.NaturalIdCacheStatistics;
-import org.hibernate.stat.SecondLevelCacheStatistics;
+import org.hibernate.cfg.*;
+import org.hibernate.exception.*;
+import org.hibernate.service.*;
+import org.hibernate.stat.*;
 
 import javax.persistence.*;
 import java.util.*;
-import java.util.concurrent.Callable;
+import java.util.concurrent.*;
 
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 import static org.apache.ignite.cache.hibernate.HibernateRegionFactory.*;
 import static org.hibernate.cfg.Environment.*;
 
+import org.apache.ignite.cache.hibernate.HibernateL2CacheSelfTest.Entity;
+
 /**
  *
  * Tests Hibernate L2 cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb8dcf9c/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
index 6f47d3b..efc6f3e 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
@@ -17,24 +17,23 @@
 
 package org.apache.ignite.cache.hibernate;
 
-import org.apache.commons.dbcp.managed.BasicManagedDataSource;
-import org.apache.ignite.cache.jta.CacheTmLookup;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.h2.jdbcx.JdbcDataSource;
-import org.hibernate.cache.spi.access.AccessType;
-import org.hibernate.engine.transaction.internal.jta.JtaTransactionFactory;
+import org.apache.commons.dbcp.managed.*;
+import org.apache.ignite.cache.jta.*;
+import org.apache.ignite.configuration.*;
+import org.h2.jdbcx.*;
+import org.hibernate.cache.spi.access.*;
+import org.hibernate.engine.transaction.internal.jta.*;
 import org.hibernate.engine.transaction.spi.TransactionFactory;
-import org.hibernate.service.ServiceRegistryBuilder;
-import org.hibernate.service.jdbc.connections.internal.DatasourceConnectionProviderImpl;
-import org.hibernate.service.jdbc.connections.spi.ConnectionProvider;
-import org.hibernate.service.jta.platform.internal.AbstractJtaPlatform;
-import org.hibernate.service.jta.platform.spi.JtaPlatform;
-import org.jetbrains.annotations.Nullable;
-import org.objectweb.jotm.Jotm;
-
-import javax.transaction.TransactionManager;
-import javax.transaction.UserTransaction;
-import java.util.Collections;
+import org.hibernate.service.*;
+import org.hibernate.service.jdbc.connections.internal.*;
+import org.hibernate.service.jdbc.connections.spi.*;
+import org.hibernate.service.jta.platform.internal.*;
+import org.hibernate.service.jta.platform.spi.*;
+import org.jetbrains.annotations.*;
+import org.objectweb.jotm.*;
+
+import javax.transaction.*;
+import java.util.*;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb8dcf9c/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
index c5a94bc..2a0d46d 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/CacheJtaManager.java
@@ -17,20 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.jta;
 
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.jta.CacheTmLookup;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.TransactionConfiguration;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
-import org.jetbrains.annotations.Nullable;
-
-import javax.transaction.RollbackException;
-import javax.transaction.SystemException;
-import javax.transaction.Transaction;
-import javax.transaction.TransactionManager;
-import java.lang.ref.WeakReference;
-import java.util.Map;
-import java.util.WeakHashMap;
+import org.apache.ignite.*;
+import org.apache.ignite.cache.jta.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
+import org.jetbrains.annotations.*;
+
+import javax.transaction.*;
+import java.lang.ref.*;
+import java.util.*;
 
 /**
  * Implementation of {@link CacheJtaManagerAdapter}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb8dcf9c/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index 9b8cb15..e1c8a77 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -17,21 +17,18 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.jta.CacheTmLookup;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.jta.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.transactions.Transaction;
-import org.objectweb.jotm.Jotm;
+import org.objectweb.jotm.*;
 
-import javax.transaction.Status;
-import javax.transaction.TransactionManager;
-import javax.transaction.UserTransaction;
+import javax.transaction.*;
 
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.transactions.TransactionState.ACTIVE;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.transactions.TransactionState.*;
 
 /**
  * Abstract class for cache tests.


[28/50] incubator-ignite git commit: #IGNITE-591 - Minor

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

Branch: refs/heads/ignite-1085
Commit: cd01e4c39362e5c27fe749fcd1cf2788b50bcc3d
Parents: 9a51cdf
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Tue Jul 7 19:21:15 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Tue Jul 7 19:21:15 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheContext.java   | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cd01e4c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index c454da9..eb813c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -220,7 +220,6 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param dataStructuresMgr Cache dataStructures manager.
      * @param ttlMgr TTL manager.
      * @param drMgr Data center replication manager.
-     * @param jtaMgr JTA manager.
      * @param rslvrMgr Conflict resolution manager.
      * @param pluginMgr Cache plugin manager.
      */


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

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


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

Branch: refs/heads/ignite-1085
Commit: 3089ace6c9de37765b6f4fa0697b1430f4cb417e
Parents: d27453f f72b291
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Sun Jul 5 11:55:10 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Sun Jul 5 11:55:10 2015 -0700

----------------------------------------------------------------------
 bin/ignite.bat                                  |   8 +-
 bin/ignite.sh                                   |   6 +-
 bin/include/parseargs.bat                       |   1 +
 bin/include/parseargs.sh                        |   3 +
 .../startup/cmdline/CommandLineStartup.java     |   3 +-
 .../startup/cmdline/CommandLineTransformer.java |   9 +
 .../GridCachePartitionedFailoverSelfTest.java   |   5 +
 modules/kafka/pom.xml                           | 116 ++++++
 .../ignite/stream/kafka/KafkaStreamer.java      | 220 +++++++++++
 .../kafka/IgniteKafkaStreamerSelfTestSuite.java |  37 ++
 .../stream/kafka/KafkaEmbeddedBroker.java       | 378 +++++++++++++++++++
 .../kafka/KafkaIgniteStreamerSelfTest.java      | 227 +++++++++++
 .../ignite/stream/kafka/SimplePartitioner.java  |  53 +++
 .../util/spring/IgniteSpringHelperImpl.java     |  72 +++-
 .../IgniteExcludeInConfigurationTest.java       |  78 ++++
 .../org/apache/ignite/spring/sprint-exclude.xml |  57 +++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 .../ignite/visor/commands/VisorConsole.scala    |   3 +-
 .../visor/commands/open/VisorOpenCommand.scala  | 319 ++++++++++++++++
 .../scala/org/apache/ignite/visor/visor.scala   | 230 +----------
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |   2 +
 .../commands/kill/VisorKillCommandSpec.scala    |   1 +
 .../commands/start/VisorStartCommandSpec.scala  |   1 +
 .../commands/tasks/VisorTasksCommandSpec.scala  |   1 +
 .../commands/vvm/VisorVvmCommandSpec.scala      |   1 +
 pom.xml                                         |   1 +
 26 files changed, 1593 insertions(+), 241 deletions(-)
----------------------------------------------------------------------



[43/50] incubator-ignite git commit: release notes

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


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

Branch: refs/heads/ignite-1085
Commit: 064d079ce4f7cc501936b95a1eaae144c6b823bb
Parents: fa5cb91
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:51:33 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:51:33 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/064d079c/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 3a5043e..bcfed27 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -1,6 +1,16 @@
 Apache Ignite Release Notes
 ===========================
 
+Apache Ignite In-Memory Data Fabric 1.3
+---------------------------------------
+
+* Added auto-retries for cache operations in recoverable cases.
+* Fixed several issues with JTA integration.
+* Stability fixes for TCP discovery SPI.
+* Stability fixes for onheap and offheap SQL queries.
+* Bug fixes in In-Memory Accelerator For Apache Hadoop.
+* Many stability and fault-tolerance fixes.
+
 Apache Ignite In-Memory Data Fabric 1.2
 ---------------------------------------
 


[32/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/82f49921
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/82f49921
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/82f49921

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

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


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

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


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

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


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

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

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


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


[09/50] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by sb...@apache.org.
IGNITE-1026 - Count down latch 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/389bad8d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/389bad8d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/389bad8d

Branch: refs/heads/ignite-1085
Commit: 389bad8d1cac892b01c0a5e5e9479337410c1109
Parents: fe92bfe
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 24 18:47:42 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 24 18:47:42 2015 -0700

----------------------------------------------------------------------
 .../processors/datastructures/DataStructuresProcessor.java  | 9 ---------
 .../datastructures/GridCacheCountDownLatchImpl.java         | 3 ---
 2 files changed, 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/389bad8d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index c26e14d..4d2ecbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.transactions.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
@@ -1158,14 +1157,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
-                                try {
-                                    removeCountDownLatch(latch0.name());
-                                }
-                                catch (IgniteCheckedException e) {
-                                    U.error(log, "Failed to automatically delete count down latch: " +
-                                        latch0.name(), e);
-                                }
-
                                 dsMap.remove(key);
 
                                 latch.onRemoved();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/389bad8d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index 33547d9..a5353d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -288,9 +288,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
     /** {@inheritDoc} */
     @Override public void close() {
-        if (rmvd)
-            return;
-
         try {
             ctx.kernalContext().dataStructures().removeCountDownLatch(name);
         }


[26/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/bc297978
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/bc297978
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/bc297978

Branch: refs/heads/ignite-1085
Commit: bc2979788d7f728766e5b70314ae45c62c9596b2
Parents: 25c349f
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jul 7 16:51:30 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jul 7 16:51:30 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/loadtests/hashmap/GridCacheTestContext.java | 1 -
 .../ignite/cache/hibernate/HibernateL2CacheSelfTest.java      | 2 --
 .../main/java/org/apache/ignite/cache/jta/CacheTmLookup.java  | 2 +-
 .../processors/cache/GridTmLookupLifecycleAwareSelfTest.java  | 7 +++----
 4 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bc297978/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index f0d1af8..e9d7a45 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -19,7 +19,6 @@ package org.apache.ignite.loadtests.hashmap;
 
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.datastructures.*;
 import org.apache.ignite.internal.processors.cache.dr.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bc297978/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
index 91d5c26..c819ec9 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
@@ -46,8 +46,6 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 import static org.apache.ignite.cache.hibernate.HibernateRegionFactory.*;
 import static org.hibernate.cfg.Environment.*;
 
-import org.apache.ignite.cache.hibernate.HibernateL2CacheSelfTest.Entity;
-
 /**
  *
  * Tests Hibernate L2 cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bc297978/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java b/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
index 5a38576..f16d6d2 100644
--- a/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
+++ b/modules/jta/src/main/java/org/apache/ignite/cache/jta/CacheTmLookup.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.cache.jta;
 
 import org.apache.ignite.*;
-import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.configuration.*;
 import org.jetbrains.annotations.*;
 
 import javax.transaction.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bc297978/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
index 261765b..f7d3902 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
@@ -44,21 +44,20 @@ public class GridTmLookupLifecycleAwareSelfTest extends GridAbstractLifecycleAwa
 
     /** */
     private boolean configureGlobalTmLookup;
-    
+
     /**
      */
     @SuppressWarnings("PublicInnerClass")
     public static class TestTxLookup extends GridAbstractLifecycleAwareSelfTest.TestLifecycleAware
         implements CacheTmLookup {
-
         /** */
         @IgniteInstanceResource
         Ignite ignite;
-        
+
         /** {@inheritDoc} */
         @Override public void start() {
             super.start();
-            
+
             assertNotNull(ignite);
         }
 


[49/50] incubator-ignite git commit: # Hive version changed in tests: 1.2.0 -> 1.2.1

Posted by sb...@apache.org.
# Hive version changed in tests: 1.2.0 -> 1.2.1


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

Branch: refs/heads/ignite-1085
Commit: bee6f6884ba0da9b9418842f7926d1dbd3ddd624
Parents: 546d595
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Jul 9 15:05:01 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Thu Jul 9 15:05:01 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bee6f688/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 2ab3e8c..7393f69 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -130,7 +130,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
      * @throws Exception If failed.
      */
     public static void downloadHive() throws Exception {
-        String ver = IgniteSystemProperties.getString("hive.version", "1.2.0");
+        String ver = IgniteSystemProperties.getString("hive.version", "1.2.1");
 
         X.println("Will use Hive version: " + ver);
 


[48/50] incubator-ignite git commit: # ignite-gg-10416 Fixed tests.

Posted by sb...@apache.org.
# ignite-gg-10416 Fixed tests.


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

Branch: refs/heads/ignite-1085
Commit: 546d5955a1fdb4a16c186242945d4a27ba13c52c
Parents: d04c104
Author: Andrey <an...@gridgain.com>
Authored: Thu Jul 9 17:14:10 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Thu Jul 9 17:14:10 2015 +0700

----------------------------------------------------------------------
 .../util/spring/IgniteSpringHelperImpl.java      | 10 +++++-----
 .../spring/IgniteExcludeInConfigurationTest.java |  5 ++++-
 .../org/apache/ignite/spring/sprint-exclude.xml  | 19 +++++++++++++++++++
 3 files changed, 28 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/546d5955/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java b/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
index 6cfca36..435f522 100644
--- a/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
+++ b/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
@@ -422,6 +422,8 @@ public class IgniteSpringHelperImpl implements IgniteSpringHelper {
         GenericApplicationContext springCtx = new GenericApplicationContext();
 
         if (excludedProps.length > 0) {
+            final List<String> excludedPropsList = Arrays.asList(excludedProps);
+
             BeanFactoryPostProcessor postProc = new BeanFactoryPostProcessor() {
                 /**
                  * @param def Registered BeanDefinition.
@@ -433,12 +435,10 @@ public class IgniteSpringHelperImpl implements IgniteSpringHelper {
                     while (iterVals.hasNext()) {
                         PropertyValue val = iterVals.next();
 
-                        for (String excludedProp : excludedProps) {
-                            if (val.getName().equals(excludedProp)) {
-                                iterVals.remove();
+                        if (excludedPropsList.contains(val.getName())) {
+                            iterVals.remove();
 
-                                return;
-                            }
+                            continue;
                         }
 
                         if (val.getValue() instanceof Iterable) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/546d5955/modules/spring/src/test/java/org/apache/ignite/spring/IgniteExcludeInConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/IgniteExcludeInConfigurationTest.java b/modules/spring/src/test/java/org/apache/ignite/spring/IgniteExcludeInConfigurationTest.java
index 1edca77..b708f21 100644
--- a/modules/spring/src/test/java/org/apache/ignite/spring/IgniteExcludeInConfigurationTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/IgniteExcludeInConfigurationTest.java
@@ -40,7 +40,8 @@ public class IgniteExcludeInConfigurationTest extends GridCommonAbstractTest {
     public void testExclude() throws Exception {
          IgniteSpringHelper spring = SPRING.create(false);
 
-        Collection<IgniteConfiguration> cfgs = spring.loadConfigurations(cfgLocation, "typeMetadata").get1();
+        Collection<IgniteConfiguration> cfgs = spring.loadConfigurations(cfgLocation, "fileSystemConfiguration",
+            "typeMetadata").get1();
 
         assertNotNull(cfgs);
         assertEquals(1, cfgs.size());
@@ -50,6 +51,8 @@ public class IgniteExcludeInConfigurationTest extends GridCommonAbstractTest {
         assertEquals(1, cfg.getCacheConfiguration().length);
         assertNull(cfg.getCacheConfiguration()[0].getTypeMetadata());
 
+        assertNull(cfg.getFileSystemConfiguration());
+
         cfgs = spring.loadConfigurations(cfgLocation, "keyType").get1();
 
         assertNotNull(cfgs);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/546d5955/modules/spring/src/test/java/org/apache/ignite/spring/sprint-exclude.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/sprint-exclude.xml b/modules/spring/src/test/java/org/apache/ignite/spring/sprint-exclude.xml
index 494f786..e6bf426 100644
--- a/modules/spring/src/test/java/org/apache/ignite/spring/sprint-exclude.xml
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/sprint-exclude.xml
@@ -29,6 +29,25 @@
         http://www.springframework.org/schema/util
         http://www.springframework.org/schema/util/spring-util.xsd">
     <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="fileSystemConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
+                    <property name="name" value="test"/>
+                    <property name="metaCacheName" value="meta"/>
+                    <property name="dataCacheName" value="data"/>
+
+                    <property name="maxSpaceSize" value="#{100L * 1024 * 1024}"/>
+
+                    <!-- Loopback endpoint. -->
+                    <property name="ipcEndpointConfiguration">
+                        <bean class="org.apache.ignite.igfs.IgfsIpcEndpointConfiguration">
+                            <property name="type" value="TCP" />
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
         <!-- Cache configurations (all properties are optional). -->
         <property name="cacheConfiguration">
             <list>


[39/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-591'

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


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

Branch: refs/heads/ignite-1085
Commit: 23dc8fc87352bdb38a0ff7d01dab048c22ceae77
Parents: 6d6ec77 cfeec2d
Author: sevdokimov <se...@gridgain.com>
Authored: Wed Jul 8 15:57:06 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed Jul 8 15:57:06 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  4 ++
 .../configuration/TransactionConfiguration.java | 23 +++++++
 .../processors/cache/GridCacheAttributes.java   |  3 +
 .../processors/cache/GridCacheContext.java      |  8 +--
 .../processors/cache/GridCacheProcessor.java    | 21 +++---
 .../cache/GridCacheSharedContext.java           | 15 +++-
 .../cache/jta/CacheJtaManagerAdapter.java       | 17 +++--
 .../cache/jta/CacheNoopJtaManager.java          |  2 +-
 .../visor/cache/VisorCacheConfiguration.java    | 11 ---
 .../loadtests/hashmap/GridCacheTestContext.java |  4 +-
 .../HibernateTransactionalDataRegion.java       | 12 +++-
 .../hibernate/HibernateL2CacheSelfTest.java     |  7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |  5 --
 .../apache/ignite/cache/jta/CacheTmLookup.java  |  3 +-
 .../processors/cache/jta/CacheJtaManager.java   | 72 ++++++++++++++++++--
 .../cache/jta/GridCacheXAResource.java          | 16 ++---
 .../processors/cache/GridCacheJtaSelfTest.java  | 52 ++++++++++----
 .../GridTmLookupLifecycleAwareSelfTest.java     | 29 ++++++--
 .../commands/cache/VisorCacheCommand.scala      |  2 -
 19 files changed, 211 insertions(+), 95 deletions(-)
----------------------------------------------------------------------



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

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


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

Branch: refs/heads/ignite-1085
Commit: c134dcfa5e7bb5dbc7a533f3d047e6e40cf2ce4e
Parents: 3089ace 0a569b8
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 8 12:20:13 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 8 12:20:13 2015 -0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  12 ++
 dev-tools/slurp.sh                              |   2 +-
 modules/core/pom.xml                            |   4 +-
 .../cache/eviction/fifo/FifoEvictionPolicy.java |   5 -
 .../cache/eviction/lru/LruEvictionPolicy.java   |   5 -
 .../eviction/sorted/SortedEvictionPolicy.java   |  19 +-
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../apache/ignite/internal/IgniteKernal.java    |  10 +-
 .../internal/interop/InteropIgnition.java       |  48 +++--
 .../internal/interop/InteropProcessor.java      |   7 +
 .../processors/cache/CacheObjectImpl.java       |   1 -
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheIoManager.java    |   8 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../dht/GridPartitionedGetFuture.java           |  13 +-
 .../distributed/near/GridNearGetFuture.java     |   4 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |  35 +++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  46 +++++
 .../tcp/internal/TcpDiscoveryNode.java          |   2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   8 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |  10 +-
 .../cache/CacheFutureExceptionSelfTest.java     | 158 +++++++++++++++
 .../GridCachePartitionedNodeRestartTest.java    |   5 -
 ...ePartitionedOptimisticTxNodeRestartTest.java |   2 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 -
 .../GridCacheReplicatedNodeRestartSelfTest.java |   5 -
 ...acheAtomicReplicatedNodeRestartSelfTest.java |  14 +-
 ...heConcurrentEvictionConsistencySelfTest.java |  15 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   4 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  38 ++++
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |  80 ++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   5 +
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 -
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 +++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 +++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 +++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 +++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ---
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 +++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 +++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ---
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   |  72 ++++++-
 .../cache/jta/GridCacheXAResource.java          |  16 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |  52 +++--
 .../GridTmLookupLifecycleAwareSelfTest.java     |  29 ++-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/kafka/pom.xml                           |  11 -
 modules/mesos/pom.xml                           |   1 -
 modules/rest-http/pom.xml                       |  14 +-
 modules/urideploy/pom.xml                       |   8 +-
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 modules/web/pom.xml                             |   6 +-
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 modules/yardstick/config/ignite-base-config.xml |  23 +++
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  55 +++++
 .../ignite/yardstick/cache/model/Person2.java   |  67 ++++++
 .../ignite/yardstick/cache/model/Person8.java   | 109 ++++++++++
 parent/pom.xml                                  |   1 +
 scripts/git-patch-prop.sh                       |   2 +-
 75 files changed, 1695 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c134dcfa/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

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


[36/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-591

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


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

Branch: refs/heads/ignite-1085
Commit: cfeec2ddb63770dfe28176c3916630753162631c
Parents: 839fe79 44b52b4
Author: sevdokimov <se...@gridgain.com>
Authored: Wed Jul 8 14:32:52 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed Jul 8 14:32:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  20 ++
 .../tcp/internal/TcpDiscoveryNode.java          |   2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   8 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |  10 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java |   8 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  38 ++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   3 +
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 modules/yardstick/config/ignite-base-config.xml |  23 +++
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  55 +++++
 .../ignite/yardstick/cache/model/Person2.java   |  67 ++++++
 .../ignite/yardstick/cache/model/Person8.java   | 109 ++++++++++
 16 files changed, 727 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



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

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


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

Branch: refs/heads/ignite-1085
Commit: 5dc2d2bc41bf04d0b7c538dd1a37b3cbc67c45e6
Parents: e54f572 44506c3
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jul 7 14:23:41 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jul 7 14:23:41 2015 +0300

----------------------------------------------------------------------
 modules/core/pom.xml                            |   4 +-
 .../internal/interop/InteropIgnition.java       |  31 ++--
 .../processors/cache/CacheObjectImpl.java       |   1 -
 .../dht/GridPartitionedGetFuture.java           |  13 +-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  26 +++
 .../cache/CacheFutureExceptionSelfTest.java     | 171 +++++++++++++++++++
 .../GridCachePartitionedNodeRestartTest.java    |   5 -
 ...ePartitionedOptimisticTxNodeRestartTest.java |   2 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 -
 .../GridCacheReplicatedNodeRestartSelfTest.java |   5 -
 ...acheAtomicReplicatedNodeRestartSelfTest.java |  10 ++
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |  80 +++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   2 +
 modules/mesos/pom.xml                           |   1 -
 modules/rest-http/pom.xml                       |  14 +-
 modules/urideploy/pom.xml                       |   8 +-
 modules/web/pom.xml                             |   6 +-
 parent/pom.xml                                  |   1 +
 20 files changed, 343 insertions(+), 49 deletions(-)
----------------------------------------------------------------------



[04/50] incubator-ignite git commit: IGNITE-1026 - Data structures fix

Posted by sb...@apache.org.
IGNITE-1026 - Data structures 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/431a62e9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/431a62e9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/431a62e9

Branch: refs/heads/ignite-1085
Commit: 431a62e99d0e853d2e4cb412d6599cbfe3407fe1
Parents: 1419d39
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 18:11:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 18:11:21 2015 -0700

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java |  11 +-
 .../IgniteClientDataStructuresAbstractTest.java | 109 ++++++++++++++-----
 2 files changed, 93 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/431a62e9/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 26e2f6c..f299a69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -819,7 +819,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         CacheConfiguration newCfg = cacheConfiguration(cfg, cacheName);
 
-        ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false, true).get();
+        if (ctx.cache().cache(cacheName) == null)
+            ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false, true).get();
 
         return cacheName;
     }
@@ -1179,6 +1180,14 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
+                                try {
+                                    removeCountDownLatch(latch0.name());
+                                }
+                                catch (IgniteCheckedException e) {
+                                    U.error(log, "Failed to automatically delete count down latch: " +
+                                        latch0.name(), e);
+                                }
+
                                 dsMap.remove(key);
 
                                 latch.onRemoved();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/431a62e9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 5a6be8e..bcfb713 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -77,12 +77,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testSequence() throws Exception {
         Ignite clientNode = clientIgnite();
-
         Ignite srvNode = serverNode();
 
-        assertNull(clientNode.atomicSequence("seq1", 1L, false));
+        testSequence(clientNode, srvNode);
+        testSequence(srvNode, clientNode);
+    }
 
-        try (IgniteAtomicSequence seq = clientNode.atomicSequence("seq1", 1L, true)) {
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testSequence(Ignite creator, Ignite other) throws Exception {
+        assertNull(creator.atomicSequence("seq1", 1L, false));
+        assertNull(other.atomicSequence("seq1", 1L, false));
+
+        try (IgniteAtomicSequence seq = creator.atomicSequence("seq1", 1L, true)) {
             assertNotNull(seq);
 
             assertEquals(1L, seq.get());
@@ -91,13 +101,13 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertEquals(2L, seq.get());
 
-            IgniteAtomicSequence seq0 = srvNode.atomicSequence("seq1", 1L, false);
+            IgniteAtomicSequence seq0 = other.atomicSequence("seq1", 1L, false);
 
             assertNotNull(seq0);
         }
 
-        assertNull(clientNode.atomicSequence("seq1", 1L, false));
-        assertNull(srvNode.atomicSequence("seq1", 1L, false));
+        assertNull(creator.atomicSequence("seq1", 1L, false));
+        assertNull(other.atomicSequence("seq1", 1L, false));
     }
 
     /**
@@ -105,12 +115,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testAtomicLong() throws Exception {
         Ignite clientNode = clientIgnite();
-
         Ignite srvNode = serverNode();
 
-        assertNull(clientNode.atomicLong("long1", 1L, false));
+        testAtomicLong(clientNode, srvNode);
+        testAtomicLong(srvNode, clientNode);
+    }
 
-        try (IgniteAtomicLong cntr = clientNode.atomicLong("long1", 1L, true)) {
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testAtomicLong(Ignite creator, Ignite other) throws Exception {
+        assertNull(creator.atomicLong("long1", 1L, false));
+        assertNull(other.atomicLong("long1", 1L, false));
+
+        try (IgniteAtomicLong cntr = creator.atomicLong("long1", 1L, true)) {
             assertNotNull(cntr);
 
             assertEquals(1L, cntr.get());
@@ -119,7 +139,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertEquals(2L, cntr.get());
 
-            IgniteAtomicLong cntr0 = srvNode.atomicLong("long1", 1L, false);
+            IgniteAtomicLong cntr0 = other.atomicLong("long1", 1L, false);
 
             assertNotNull(cntr0);
 
@@ -130,8 +150,8 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
             assertEquals(3L, cntr.get());
         }
 
-        assertNull(clientNode.atomicLong("long1", 1L, false));
-        assertNull(srvNode.atomicLong("long1", 1L, false));
+        assertNull(creator.atomicLong("long1", 1L, false));
+        assertNull(other.atomicLong("long1", 1L, false));
     }
 
     /**
@@ -139,14 +159,24 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testSet() throws Exception {
         Ignite clientNode = clientIgnite();
-
         Ignite srvNode = serverNode();
 
-        assertNull(clientNode.set("set1", null));
+        testSet(clientNode, srvNode);
+        testSet(srvNode, clientNode);
+    }
+
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testSet(Ignite creator, Ignite other) throws Exception {
+        assertNull(creator.set("set1", null));
+        assertNull(other.set("set1", null));
 
         CollectionConfiguration colCfg = new CollectionConfiguration();
 
-        try (IgniteSet<Integer> set = clientNode.set("set1", colCfg)) {
+        try (IgniteSet<Integer> set = creator.set("set1", colCfg)) {
             assertNotNull(set);
 
             assertEquals(0, set.size());
@@ -157,7 +187,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertTrue(set.contains(1));
 
-            IgniteSet<Integer> set0 = srvNode.set("set1", null);
+            IgniteSet<Integer> set0 = other.set("set1", null);
 
             assertTrue(set0.contains(1));
 
@@ -167,6 +197,9 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertFalse(set.contains(1));
         }
+
+        assertNull(creator.set("set1", null));
+        assertNull(other.set("set1", null));
     }
 
     /**
@@ -174,12 +207,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testLatch() throws Exception {
         Ignite clientNode = clientIgnite();
+        Ignite srvNode = serverNode();
 
-        final Ignite srvNode = serverNode();
+        testLatch(clientNode, srvNode);
+        testLatch(srvNode, clientNode);
+    }
 
-        assertNull(clientNode.countDownLatch("latch1", 1, true, false));
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testLatch(Ignite creator, final Ignite other) throws Exception {
+        assertNull(creator.countDownLatch("latch1", 1, true, false));
+        assertNull(other.countDownLatch("latch1", 1, true, false));
 
-        try (IgniteCountDownLatch latch = clientNode.countDownLatch("latch1", 1, true, true)) {
+        try (IgniteCountDownLatch latch = creator.countDownLatch("latch1", 1, true, true)) {
             assertNotNull(latch);
 
             assertEquals(1, latch.count());
@@ -188,7 +231,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
                 @Override public Object call() throws Exception {
                     U.sleep(1000);
 
-                    IgniteCountDownLatch latch0 = srvNode.countDownLatch("latch1", 1, true, false);
+                    IgniteCountDownLatch latch0 = other.countDownLatch("latch1", 1, true, false);
 
                     assertEquals(1, latch0.count());
 
@@ -210,6 +253,9 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             fut.get();
         }
+
+        assertNull(creator.countDownLatch("latch1", 1, true, false));
+        assertNull(other.countDownLatch("latch1", 1, true, false));
     }
 
     /**
@@ -217,14 +263,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testQueue() throws Exception {
         Ignite clientNode = clientIgnite();
+        Ignite srvNode = serverNode();
 
-        final Ignite srvNode = serverNode();
-
-        CollectionConfiguration colCfg = new CollectionConfiguration();
+        testQueue(clientNode, srvNode);
+        testQueue(srvNode, clientNode);
+    }
 
-        assertNull(clientNode.queue("q1", 0, null));
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testQueue(Ignite creator, final Ignite other) throws Exception {
+        assertNull(creator.queue("q1", 0, null));
+        assertNull(other.queue("q1", 0, null));
 
-        try (IgniteQueue<Integer> queue = clientNode.queue("q1", 0, colCfg)) {
+        try (IgniteQueue<Integer> queue = creator.queue("q1", 0, new CollectionConfiguration())) {
             assertNotNull(queue);
 
             queue.add(1);
@@ -235,7 +289,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
                 @Override public Object call() throws Exception {
                     U.sleep(1000);
 
-                    IgniteQueue<Integer> queue0 = srvNode.queue("q1", 0, null);
+                    IgniteQueue<Integer> queue0 = other.queue("q1", 0, null);
 
                     assertEquals(0, queue0.size());
 
@@ -255,6 +309,9 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             fut.get();
         }
+
+        assertNull(creator.queue("q1", 0, null));
+        assertNull(other.queue("q1", 0, null));
     }
 
     /**


[30/50] incubator-ignite git commit: # ignite-gg-10416 Exclude lifecycleBeans for daemon node.

Posted by sb...@apache.org.
# ignite-gg-10416 Exclude lifecycleBeans for daemon node.


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

Branch: refs/heads/ignite-1085
Commit: 8cc75fc562706be8aebb837b708c03a9be264027
Parents: 0f1b31a
Author: Andrey <an...@gridgain.com>
Authored: Wed Jul 8 10:50:16 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Jul 8 10:50:16 2015 +0700

----------------------------------------------------------------------
 .../org/apache/ignite/visor/commands/open/VisorOpenCommand.scala   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8cc75fc5/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
index 6498baf..632a96b 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/open/VisorOpenCommand.scala
@@ -144,7 +144,7 @@ class VisorOpenCommand extends VisorConsoleCommand {
                     try
                         // Cache, IGFS, indexing SPI configurations should be excluded from daemon node config.
                         spring.loadConfigurations(url, "cacheConfiguration", "fileSystemConfiguration",
-                            "indexingSpi").get1()
+                            "lifecycleBeans", "indexingSpi").get1()
                     finally {
                         if (log4jTup != null)
                             U.removeLog4jNoOpLogger(log4jTup)


[14/50] incubator-ignite git commit: # IGNITE-591 Enable tests.

Posted by sb...@apache.org.
# IGNITE-591 Enable tests.


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

Branch: refs/heads/ignite-1085
Commit: bac6f481dd766ac1105fcc127e7ee557f96598de
Parents: 4b06e77
Author: sevdokimov <se...@gridgain.com>
Authored: Thu Jul 2 14:56:12 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu Jul 2 14:56:12 2015 +0300

----------------------------------------------------------------------
 .../hibernate/HibernateL2CacheSelfTest.java     | 50 ++++++++++----------
 .../HibernateL2CacheTransactionalSelfTest.java  | 38 +++++++--------
 .../processors/cache/GridCacheJtaSelfTest.java  | 23 ++++-----
 3 files changed, 55 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac6f481/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
index bb5884a..fe34ea8 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheSelfTest.java
@@ -17,32 +17,37 @@
 
 package org.apache.ignite.cache.hibernate;
 
-import org.apache.ignite.cache.affinity.rendezvous.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.hibernate.*;
 import org.hibernate.Query;
-import org.hibernate.annotations.*;
-import org.hibernate.cache.spi.*;
+import org.hibernate.annotations.NaturalId;
+import org.hibernate.annotations.NaturalIdCache;
+import org.hibernate.cache.spi.GeneralDataRegion;
+import org.hibernate.cache.spi.TransactionalDataRegion;
 import org.hibernate.cache.spi.access.AccessType;
-import org.hibernate.cfg.*;
-import org.hibernate.exception.*;
-import org.hibernate.service.*;
-import org.hibernate.stat.*;
+import org.hibernate.cfg.Configuration;
+import org.hibernate.exception.ConstraintViolationException;
+import org.hibernate.service.ServiceRegistryBuilder;
+import org.hibernate.stat.NaturalIdCacheStatistics;
+import org.hibernate.stat.SecondLevelCacheStatistics;
 
 import javax.persistence.*;
 import java.util.*;
-import java.util.concurrent.*;
+import java.util.concurrent.Callable;
 
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.cache.hibernate.HibernateRegionFactory.*;
 import static org.hibernate.cfg.Environment.*;
 
@@ -51,11 +56,6 @@ import static org.hibernate.cfg.Environment.*;
  * Tests Hibernate L2 cache.
  */
 public class HibernateL2CacheSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-591");
-    }
-
     /** */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
@@ -1128,6 +1128,8 @@ public class HibernateL2CacheSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testNaturalIdCache() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1084");
+
         for (AccessType accessType : accessTypes())
             testNaturalIdCache(accessType);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac6f481/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
index c429d9a..6f47d3b 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheTransactionalSelfTest.java
@@ -17,23 +17,24 @@
 
 package org.apache.ignite.cache.hibernate;
 
-import org.apache.commons.dbcp.managed.*;
-import org.apache.ignite.cache.jta.*;
-import org.apache.ignite.configuration.*;
-import org.h2.jdbcx.*;
-import org.hibernate.cache.spi.access.*;
-import org.hibernate.engine.transaction.internal.jta.*;
+import org.apache.commons.dbcp.managed.BasicManagedDataSource;
+import org.apache.ignite.cache.jta.CacheTmLookup;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.h2.jdbcx.JdbcDataSource;
+import org.hibernate.cache.spi.access.AccessType;
+import org.hibernate.engine.transaction.internal.jta.JtaTransactionFactory;
 import org.hibernate.engine.transaction.spi.TransactionFactory;
-import org.hibernate.service.*;
-import org.hibernate.service.jdbc.connections.internal.*;
-import org.hibernate.service.jdbc.connections.spi.*;
-import org.hibernate.service.jta.platform.internal.*;
-import org.hibernate.service.jta.platform.spi.*;
-import org.jetbrains.annotations.*;
-import org.objectweb.jotm.*;
-
-import javax.transaction.*;
-import java.util.*;
+import org.hibernate.service.ServiceRegistryBuilder;
+import org.hibernate.service.jdbc.connections.internal.DatasourceConnectionProviderImpl;
+import org.hibernate.service.jdbc.connections.spi.ConnectionProvider;
+import org.hibernate.service.jta.platform.internal.AbstractJtaPlatform;
+import org.hibernate.service.jta.platform.spi.JtaPlatform;
+import org.jetbrains.annotations.Nullable;
+import org.objectweb.jotm.Jotm;
+
+import javax.transaction.TransactionManager;
+import javax.transaction.UserTransaction;
+import java.util.Collections;
 
 /**
  *
@@ -41,11 +42,6 @@ import java.util.*;
  * to used the same TransactionManager).
  */
 public class HibernateL2CacheTransactionalSelfTest extends HibernateL2CacheSelfTest {
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-591");
-    }
-
     /** */
     private static Jotm jotm;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bac6f481/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index 37e9ed2..9b8cb15 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -17,18 +17,21 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.jta.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.jta.CacheTmLookup;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.transactions.Transaction;
-import org.objectweb.jotm.*;
+import org.objectweb.jotm.Jotm;
 
-import javax.transaction.*;
+import javax.transaction.Status;
+import javax.transaction.TransactionManager;
+import javax.transaction.UserTransaction;
 
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.transactions.TransactionState.*;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.transactions.TransactionState.ACTIVE;
 
 /**
  * Abstract class for cache tests.
@@ -145,8 +148,6 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
     }
 
     /**
-     * TODO: IGNITE-591.
-     *
      * @throws Exception If failed.
      */
     @SuppressWarnings("ConstantConditions")


[10/50] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by sb...@apache.org.
IGNITE-1026 - Count down latch 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/7c935a58
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/7c935a58
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/7c935a58

Branch: refs/heads/ignite-1085
Commit: 7c935a58ab6cc37ac062f1c37a2650504e9ef95c
Parents: 389bad8
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 26 14:30:16 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 26 14:30:16 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c935a58/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 6c7c511..60ffce0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2421,9 +2421,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /**
      * @param cache Cache.
      */
-    private void checkNearCacheStarted(IgniteCacheProxy<?, ?> cache) {
+    private void checkNearCacheStarted(IgniteCacheProxy<?, ?> cache) throws IgniteCheckedException {
         if (!cache.context().isNear())
-            throw new IgniteException("Failed to start near cache " +
+            throw new IgniteCheckedException("Failed to start near cache " +
                 "(a cache with the same name without near cache is already started)");
     }